You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/03/14 03:40:19 UTC

[GitHub] [spark] dcoliversun commented on a change in pull request #35823: [SPARK-38533][SQL] DS V2 aggregate push-down supports project with alias

dcoliversun commented on a change in pull request #35823:
URL: https://github.com/apache/spark/pull/35823#discussion_r825562687



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -779,15 +779,19 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
     checkAnswer(df, Seq(Row(1d), Row(1d), Row(null)))
   }
 
-  test("scan with aggregate push-down: aggregate over alias NOT push down") {
+  test("scan with aggregate push-down: aggregate over alias push down") {

Review comment:
       Hi. Is it better to specify `SPARK-38533` ?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -1032,4 +1036,76 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
         |ON h2.test.view1.`|col1` = h2.test.view2.`|col1`""".stripMargin)
     checkAnswer(df, Seq.empty[Row])
   }
+
+  test("scan with aggregate push-down: complete push-down aggregate with alias") {
+    val df = spark.table("h2.test.employee")
+      .select($"DEPT", $"SALARY".as("mySalary"))
+      .groupBy($"DEPT")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =

Review comment:
       Why not use `camel case` naming? https://docs.scala-lang.org/style/naming-conventions.html

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -1032,4 +1036,76 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
         |ON h2.test.view1.`|col1` = h2.test.view2.`|col1`""".stripMargin)
     checkAnswer(df, Seq.empty[Row])
   }
+
+  test("scan with aggregate push-down: complete push-down aggregate with alias") {

Review comment:
       ditto

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -1032,4 +1036,76 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
         |ON h2.test.view1.`|col1` = h2.test.view2.`|col1`""".stripMargin)
     checkAnswer(df, Seq.empty[Row])
   }
+
+  test("scan with aggregate push-down: complete push-down aggregate with alias") {
+    val df = spark.table("h2.test.employee")
+      .select($"DEPT", $"SALARY".as("mySalary"))
+      .groupBy($"DEPT")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df, expected_plan_fragment)
+    }
+    checkAnswer(df, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+
+    val df2 = spark.table("h2.test.employee")
+      .select($"DEPT".as("myDept"), $"SALARY".as("mySalary"))
+      .groupBy($"myDept")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df2)
+    df2.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df2, expected_plan_fragment)
+    }
+    checkAnswer(df2, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+  }
+
+  test("scan with aggregate push-down: partial push-down aggregate with alias") {

Review comment:
       ditto

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -1032,4 +1036,76 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
         |ON h2.test.view1.`|col1` = h2.test.view2.`|col1`""".stripMargin)
     checkAnswer(df, Seq.empty[Row])
   }
+
+  test("scan with aggregate push-down: complete push-down aggregate with alias") {
+    val df = spark.table("h2.test.employee")
+      .select($"DEPT", $"SALARY".as("mySalary"))
+      .groupBy($"DEPT")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df, expected_plan_fragment)
+    }
+    checkAnswer(df, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+
+    val df2 = spark.table("h2.test.employee")
+      .select($"DEPT".as("myDept"), $"SALARY".as("mySalary"))
+      .groupBy($"myDept")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df2)
+    df2.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df2, expected_plan_fragment)
+    }
+    checkAnswer(df2, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+  }
+
+  test("scan with aggregate push-down: partial push-down aggregate with alias") {
+    val df = spark.read
+      .option("partitionColumn", "DEPT")
+      .option("lowerBound", "0")
+      .option("upperBound", "2")
+      .option("numPartitions", "2")
+      .table("h2.test.employee")
+      .select($"NAME", $"SALARY".as("mySalary"))
+      .groupBy($"NAME")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df, false)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =

Review comment:
       ditto

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -1032,4 +1036,76 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
         |ON h2.test.view1.`|col1` = h2.test.view2.`|col1`""".stripMargin)
     checkAnswer(df, Seq.empty[Row])
   }
+
+  test("scan with aggregate push-down: complete push-down aggregate with alias") {
+    val df = spark.table("h2.test.employee")
+      .select($"DEPT", $"SALARY".as("mySalary"))
+      .groupBy($"DEPT")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df, expected_plan_fragment)
+    }
+    checkAnswer(df, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+
+    val df2 = spark.table("h2.test.employee")
+      .select($"DEPT".as("myDept"), $"SALARY".as("mySalary"))
+      .groupBy($"myDept")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df2)
+    df2.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =

Review comment:
       ditto

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
##########
@@ -1032,4 +1036,76 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
         |ON h2.test.view1.`|col1` = h2.test.view2.`|col1`""".stripMargin)
     checkAnswer(df, Seq.empty[Row])
   }
+
+  test("scan with aggregate push-down: complete push-down aggregate with alias") {
+    val df = spark.table("h2.test.employee")
+      .select($"DEPT", $"SALARY".as("mySalary"))
+      .groupBy($"DEPT")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df, expected_plan_fragment)
+    }
+    checkAnswer(df, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+
+    val df2 = spark.table("h2.test.employee")
+      .select($"DEPT".as("myDept"), $"SALARY".as("mySalary"))
+      .groupBy($"myDept")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df2)
+    df2.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]"
+        checkKeywordsExistsInExplain(df2, expected_plan_fragment)
+    }
+    checkAnswer(df2, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00)))
+  }
+
+  test("scan with aggregate push-down: partial push-down aggregate with alias") {
+    val df = spark.read
+      .option("partitionColumn", "DEPT")
+      .option("lowerBound", "0")
+      .option("upperBound", "2")
+      .option("numPartitions", "2")
+      .table("h2.test.employee")
+      .select($"NAME", $"SALARY".as("mySalary"))
+      .groupBy($"NAME")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df, false)
+    df.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =
+          "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [NAME]"
+        checkKeywordsExistsInExplain(df, expected_plan_fragment)
+    }
+    checkAnswer(df, Seq(Row("alex", 12000.00), Row("amy", 10000.00),
+      Row("cathy", 9000.00), Row("david", 10000.00), Row("jen", 12000.00)))
+
+    val df2 = spark.read
+      .option("partitionColumn", "DEPT")
+      .option("lowerBound", "0")
+      .option("upperBound", "2")
+      .option("numPartitions", "2")
+      .table("h2.test.employee")
+      .select($"NAME".as("myName"), $"SALARY".as("mySalary"))
+      .groupBy($"myName")
+      .agg(sum($"mySalary").as("total"))
+      .filter($"total" > 1000)
+    checkAggregateRemoved(df2, false)
+    df2.queryExecution.optimizedPlan.collect {
+      case _: DataSourceV2ScanRelation =>
+        val expected_plan_fragment =

Review comment:
       ditto

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -92,23 +92,38 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
     // 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, filters, sHolder: ScanBuilderHolder)
-          if filters.isEmpty && project.forall(_.isInstanceOf[AttributeReference]) =>
+        case ScanOperation(project, filters, sHolder: ScanBuilderHolder) if filters.isEmpty &&
+          project.forall(p => p.isInstanceOf[AttributeReference] || p.isInstanceOf[Alias]) =>
           sHolder.builder match {
             case r: SupportsPushDownAggregates =>
+              val aliasAttrToOriginAttr = mutable.HashMap.empty[Expression, AttributeReference]
+              val originAttrToAliasAttr = mutable.HashMap.empty[Expression, Attribute]
+              collectAliases(project, aliasAttrToOriginAttr, originAttrToAliasAttr)
+              val newResultExpressions = resultExpressions.map { expr =>
+                expr.transform {
+                  case r: AttributeReference if aliasAttrToOriginAttr.contains(r.canonicalized) =>
+                    aliasAttrToOriginAttr(r.canonicalized)
+                }
+              }.asInstanceOf[Seq[NamedExpression]]
+              val newGroupingExpressions = groupingExpressions.map { expr =>
+                expr.transform {
+                  case r: AttributeReference if aliasAttrToOriginAttr.contains(r.canonicalized) =>
+                    aliasAttrToOriginAttr(r.canonicalized)

Review comment:
       These two lambda expressions behave the same, we can consider reusing a function




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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