huaxingao commented on a change in pull request #32049:
URL: https://github.com/apache/spark/pull/32049#discussion_r654143220



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,61 +17,183 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, 
NamedExpression, ProjectionOverSchema, SubqueryExpression}
-import org.apache.spark.sql.catalyst.planning.ScanOperation
-import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, 
Project}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.planning.{OperationHelper, ScanOperation}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, 
LeafNode, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.catalyst.util.toPrettySQL
+import org.apache.spark.sql.connector.expressions.Aggregation
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, 
SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper
+  with OperationHelper with PredicateHelper {
   import DataSourceV2Implicits._
 
-  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = 
relation.table.asReadable.newScanBuilder(relation.options)
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    
applyColumnPruning(pushdownAggregate(pushDownFilters(createScanBuilder(plan))))
+  }
+
+  private def createScanBuilder(plan: LogicalPlan) = plan.transform {
+    case r: DataSourceV2Relation =>
+      ScanBuilderHolder(r.output, r, 
r.table.asReadable.newScanBuilder(r.options))
+  }
+
+  private def pushDownFilters(plan: LogicalPlan) = plan.transform {
+    // update the scan builder with filter push down and return a new plan 
with filter pushed
+    case filter @ Filter(_, sHolder: ScanBuilderHolder) =>
+      val (filters, _, _) = collectFilters(filter).get
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, 
relation.output)
+      val normalizedFilters =
+        DataSourceStrategy.normalizeExprs(filters, sHolder.relation.output)
       val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
         normalizedFilters.partition(SubqueryExpression.hasSubquery)
 
       // `pushedFilters` will be pushed down and evaluated in the underlying 
data sources.
       // `postScanFilters` need to be evaluated after the scan.
       // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet 
row group filter.
       val (pushedFilters, postScanFiltersWithoutSubquery) = 
PushDownUtils.pushFilters(
-        scanBuilder, normalizedFiltersWithoutSubquery)
+        sHolder.builder, normalizedFiltersWithoutSubquery)
       val postScanFilters = postScanFiltersWithoutSubquery ++ 
normalizedFiltersWithSubquery
 
+      logInfo(
+        s"""
+           |Pushing operators to ${sHolder.relation.name}
+           |Pushed Filters: ${pushedFilters.mkString(", ")}
+           |Post-Scan Filters: ${postScanFilters.mkString(",")}
+         """.stripMargin)
+
+      val filterCondition = postScanFilters.reduceLeftOption(And)
+      filterCondition.map(Filter(_, sHolder)).getOrElse(sHolder)
+  }
+
+  def pushdownAggregate(plan: LogicalPlan): LogicalPlan = plan.transform {
+    // update the scan builder with agg pushdown and return a new plan with 
agg pushed
+    case aggNode @ Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, _, sHolder: ScanBuilderHolder) =>
+          sHolder.builder match {
+            case r: SupportsPushDownAggregates =>
+              if 
(sHolder.builder.asInstanceOf[SupportsPushDownFilters].pushedFilters().length 
<= 0
+                || r.supportsPushDownAggregateWithFilter()) {
+                if (r.supportsGlobalAggregatePushDownOnly() && 
groupingExpressions.nonEmpty) {
+                  aggNode // return original plan node
+                } else {
+                  val aggregates = getAggregateExpression(resultExpressions, 
project, sHolder)
+                  val pushedAggregates = PushDownUtils
+                    .pushAggregates(sHolder.builder, aggregates, 
groupingExpressions)
+                  if (pushedAggregates.aggregateExpressions.isEmpty) {
+                    aggNode // return original plan node
+                  } else {
+                    // use the aggregate columns as the output columns
+                    // e.g. TABLE t (c1 INT, c2 INT, c3 INT)
+                    // SELECT min(c1), max(c1) FROM t;
+                    // Use min(c1), max(c1) as output for 
DataSourceV2ScanRelation
+                    // We want to have the following logical plan:
+                    // == Optimized Logical Plan ==
+                    // Aggregate [min(min(c1)#21) AS min(c1)#17, 
max(max(c1)#22) AS max(c1)#18]
+                    // +- RelationV2[min(c1)#21, max(c1)#22] parquet file ...
+                    val output = aggregates.map {
+                      case agg: AggregateExpression =>
+                        AttributeReference(toPrettySQL(agg), agg.dataType)()
+                    }
+
+                    // No need to do column pruning because only the aggregate 
columns are used as
+                    // DataSourceV2ScanRelation output columns. All the other 
columns are not
+                    // included in the output. Since 
PushDownUtils.pruneColumns is not called,
+                    // ScanBuilder.requiredSchema is not pruned, but 
ScanBuilder.requiredSchema is
+                    // not used anyways. The schema for aggregate columns will 
be built in Scan.
+                    val scan = sHolder.builder.build()
+
+                    logInfo(
+                      s"""
+                         |Pushing operators to ${sHolder.relation.name}
+                         |Pushed Aggregate Functions:
+                         | ${pushedAggregates.aggregateExpressions.mkString(", 
")}
+                         |Output: ${output.mkString(", ")}
+                      """.stripMargin)
+
+                    val scanRelation = 
DataSourceV2ScanRelation(sHolder.relation, scan, output)
+                    val plan = Aggregate(groupingExpressions, 
resultExpressions, scanRelation)
+
+                    // Change the optimized logical plan to reflect the pushed 
down aggregate
+                    // e.g. TABLE t (c1 INT, c2 INT, c3 INT)
+                    // SELECT min(c1), max(c1) FROM t;
+                    // The original logical plan is
+                    // Aggregate [min(c1#9) AS min(c1)#17, max(c1#9) AS 
max(c1)#18]
+                    // +- RelationV2[c1#9] parquet ...
+                    //
+                    // After change the V2ScanRelation output to [min(_1)#21, 
max(_1)#22]
+                    // we have the following
+                    // !Aggregate [min(_1#9) AS min(_1)#17, max(_1#9) AS 
max(_1)#18]
+                    // +- RelationV2[min(_1)#21, max(_1)#22] parquet ...
+                    //
+                    // We want to change it to
+                    // == Optimized Logical Plan ==
+                    // Aggregate [min(min(c1)#21) AS min(c1)#17, 
max(max(c1)#22) AS max(c1)#18]

Review comment:
       Sorry, I got confused here. Suppose we have a query `SELECT Max(col) 
FROM t`. We push down Max(col) to parquet, then for each partition file, we 
read footer to get Max(col), and these are the partial aggregate results. We 
still need to do a final aggregate, so I guess we still need the aggregate on 
RelationV2?




-- 
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:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to