rdblue commented on a change in pull request #2022:
URL: https://github.com/apache/iceberg/pull/2022#discussion_r559055832



##########
File path: 
spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.spark.sql.catalyst.utils
+
+import java.util.UUID
+import org.apache.spark.sql
+import org.apache.spark.sql.{catalyst, AnalysisException, SparkSession}
+import org.apache.spark.sql.catalyst.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.{AccumulateFiles, Alias, 
Attribute, AttributeReference, GreaterThan, IcebergBucketTransform, 
IcebergDayTransform, IcebergHourTransform, IcebergMonthTransform, 
IcebergYearTransform, Literal, NamedExpression, PredicateHelper}
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, 
Complete, Sum}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, 
DynamicFileFilter, DynamicFileFilterWithCountCheck, Filter, LogicalPlan, 
Project}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.expressions.{BucketTransform, 
DaysTransform, Expression, FieldReference, HoursTransform, IdentityTransform, 
Lit, MonthsTransform, NamedReference, NamedTransform, Ref, Transform, 
YearsTransform}
+import 
org.apache.spark.sql.connector.iceberg.distributions.{ClusteredDistribution, 
Distribution, OrderedDistribution, UnspecifiedDistribution}
+import org.apache.spark.sql.connector.iceberg.expressions.{NullOrdering, 
SortDirection, SortOrder}
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, 
LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+  val ROW_ID_COL = "_row_id"
+  val AFFECTED_FILES_ACC_NAME = "affectedFiles"
+  val AFFECTED_FILES_ACC_ALIAS_NAME = "_affectedFiles_"
+  val SUM_ROW_ID_ALIAS_NAME = "_sum_"
+
+  def buildScanPlan(spark: SparkSession,
+                    table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan,
+                    performCountCheckForMerge: Boolean = false): LogicalPlan = 
{
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, 
toOutputAttrs(scan.readSchema(), output))
+
+    scan match {
+      case filterable: SupportsFileFilter =>
+        if (performCountCheckForMerge) {
+          val affectedFilesAcc = new SetAccumulator[String]()
+          spark.sparkContext.register(affectedFilesAcc, 
AFFECTED_FILES_ACC_NAME)
+          val planWithAccumulator = 
buildPlanWithFileAccumulator(affectedFilesAcc, prunedTargetPlan)
+          DynamicFileFilterWithCountCheck(scanRelation, affectedFilesAcc,
+            planWithAccumulator, filterable, table.name())
+        } else {
+          val matchingFilePlan = buildAggregatePlan(prunedTargetPlan)
+          DynamicFileFilter(scanRelation, matchingFilePlan, filterable)
+        }
+      case _ =>
+        scanRelation
+    }
+  }
+
+  private def buildAggregatePlan(prunedTargetPlan: LogicalPlan): LogicalPlan = 
{
+    val fileAttr = findOutputAttr(prunedTargetPlan, FILE_NAME_COL)
+    Aggregate(Seq(fileAttr), Seq(fileAttr), prunedTargetPlan)
+  }
+
+  private def buildPlanWithFileAccumulator( fileAccum: SetAccumulator[String],
+                                            prunedTargetPlan: LogicalPlan): 
LogicalPlan = {
+    val fileAttr = findOutputAttr(prunedTargetPlan, FILE_NAME_COL)
+    val rowIdAttr = findOutputAttr(prunedTargetPlan, ROW_ID_COL)
+    val projectList = Seq(fileAttr, rowIdAttr,
+      Alias(AccumulateFiles(fileAccum, fileAttr), 
AFFECTED_FILES_ACC_ALIAS_NAME )())
+    val projectPlan = Project(projectList, prunedTargetPlan)
+    val affectedFilesAttr = findOutputAttr(projectPlan, 
AFFECTED_FILES_ACC_ALIAS_NAME)
+    val aggSumCol =
+      Alias(AggregateExpression(Sum(affectedFilesAttr), Complete, false), 
SUM_ROW_ID_ALIAS_NAME)()
+    val aggPlan = Aggregate(Seq(rowIdAttr), Seq(aggSumCol), projectPlan)
+    val sumAttr = findOutputAttr(aggPlan, SUM_ROW_ID_ALIAS_NAME)
+    val havingExpr = GreaterThan(sumAttr, Literal(1L))
+    Filter(havingExpr, aggPlan)
+  }
+
+  def findOutputAttr(plan: LogicalPlan, attrName: String): Attribute = {
+    val resolver = SQLConf.get.resolver
+    plan.output.find(attr => resolver(attr.name, attrName)).getOrElse {
+      throw new AnalysisException(s"Cannot find $attrName in ${plan.output}")
+    }
+  }
+
+  def newWriteInfo(schema: StructType): LogicalWriteInfo = {
+    val uuid = UUID.randomUUID()
+    LogicalWriteInfoImpl(queryId = uuid.toString, schema, 
CaseInsensitiveStringMap.empty)
+  }
+
+  private def toOutputAttrs(schema: StructType, output: 
Seq[AttributeReference]): Seq[AttributeReference] = {
+    val nameToAttr = output.map(_.name).zip(output).toMap
+    schema.toAttributes.map {
+      a => nameToAttr.get(a.name) match {
+        case Some(ref) =>
+          // keep the attribute id if it was present in the relation
+          a.withExprId(ref.exprId)
+        case _ =>
+          // if the field is new, create a new attribute
+          AttributeReference(a.name, a.dataType, a.nullable, a.metadata)()
+      }
+    }
+  }
+
+  object BucketTransform {
+    def unapply(transform: Transform): Option[(Int, FieldReference)] = 
transform match {
+      case bt: BucketTransform => bt.columns match {
+        case Seq(nf: NamedReference) =>
+          Some(bt.numBuckets.value(), FieldReference(nf.fieldNames()))
+        case _ =>
+          None
+      }
+      case _ => None
+    }
+  }
+
+  def toCatalyst(distribution: Distribution,

Review comment:
       I think this should probably be `private` as well.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to