cloud-fan commented on code in PR #37320:
URL: https://github.com/apache/spark/pull/37320#discussion_r935063965


##########
sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala:
##########
@@ -864,6 +851,254 @@ class JDBCV2Suite extends QueryTest with 
SharedSparkSession with ExplainSuiteHel
     checkAnswer(df2, Seq(Row(2, "david", 10000.00)))
   }
 
+  test("scan with aggregate push-down and top N push-down") {
+    val df1 = spark.read
+      .table("h2.test.employee")
+      .groupBy("DEPT").sum("SALARY")
+      .orderBy("DEPT")
+      .limit(1)
+    checkSortRemoved(df1)
+    checkLimitRemoved(df1)
+    checkPushedInfo(df1,
+      "PushedAggregates: [SUM(SALARY)]",
+      "PushedGroupByExpressions: [DEPT]",
+      "PushedFilters: []",
+      "PushedTopN: ORDER BY [DEPT ASC NULLS FIRST] LIMIT 1")
+    checkAnswer(df1, Seq(Row(1, 19000.00)))
+
+    val df2 = spark.read
+      .table("h2.test.employee")
+      .select($"DEPT".as("my_dept"), $"SALARY")
+      .groupBy("my_dept").sum("SALARY")
+      .orderBy("my_dept")
+      .limit(1)
+    checkSortRemoved(df2)
+    checkLimitRemoved(df2)
+    checkPushedInfo(df2,
+      "PushedAggregates: [SUM(SALARY)]",
+      "PushedGroupByExpressions: [DEPT]",
+      "PushedFilters: []",
+      "PushedTopN: ORDER BY [DEPT ASC NULLS FIRST] LIMIT 1")
+    checkAnswer(df2, Seq(Row(1, 19000.00)))
+
+    val df3 = spark.read
+      .table("h2.test.employee")
+      .select($"SALARY",
+        when(($"SALARY" > 8000).and($"SALARY" < 10000), 
$"salary").otherwise(0).as("key"))
+      .groupBy("key").sum("SALARY")
+      .orderBy("key")
+      .limit(1)
+    checkSortRemoved(df3)
+    checkLimitRemoved(df3)
+    checkPushedInfo(df3,
+      "PushedAggregates: [SUM(SALARY)]",
+      "PushedGroupByExpressions: " +
+        "[CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY 
ELSE 0.00 END]",
+      "PushedFilters: []",
+      "PushedTopN: ORDER BY [" +
+        "CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 
0.00 END " +
+        "ASC NULLS FIRST] LIMIT 1")
+    checkAnswer(df3, Seq(Row(0, 44000.00)))
+
+    val df4 = spark.read
+      .table("h2.test.employee")
+      .groupBy("DEPT", "IS_MANAGER").sum("SALARY")
+      .orderBy("DEPT", "IS_MANAGER")
+      .limit(1)
+    checkSortRemoved(df4)
+    checkLimitRemoved(df4)
+    checkPushedInfo(df4,
+      "PushedAggregates: [SUM(SALARY)]",
+      "PushedGroupByExpressions: [DEPT, IS_MANAGER]",
+      "PushedFilters: []",
+      "PushedTopN: ORDER BY [DEPT ASC NULLS FIRST, IS_MANAGER ASC NULLS FIRST] 
LIMIT 1")
+    checkAnswer(df4, Seq(Row(1, false, 9000.00)))
+
+    val df5 = spark.read
+      .table("h2.test.employee")
+      .select($"DEPT".as("my_dept"), $"IS_MANAGER".as("my_manager"), $"SALARY")
+      .groupBy("my_dept", "my_manager").sum("SALARY")
+      .orderBy("my_dept", "my_manager")
+      .limit(1)
+    checkSortRemoved(df5)
+    checkLimitRemoved(df5)
+    checkPushedInfo(df5,
+      "PushedAggregates: [SUM(SALARY)]",
+      "PushedGroupByExpressions: [DEPT, IS_MANAGER]",
+      "PushedFilters: []",
+      "PushedTopN: ORDER BY [DEPT ASC NULLS FIRST, IS_MANAGER ASC NULLS FIRST] 
LIMIT 1")
+    checkAnswer(df5, Seq(Row(1, false, 9000.00)))
+
+    val df6 = spark.read
+      .table("h2.test.employee")
+      .select($"SALARY", $"IS_MANAGER",
+        when(($"SALARY" > 8000).and($"SALARY" < 10000), 
$"salary").otherwise(0).as("key"))
+      .groupBy("key", "IS_MANAGER").sum("SALARY")
+      .orderBy("key", "IS_MANAGER")
+      .limit(1)
+    checkSortRemoved(df6)
+    checkLimitRemoved(df6)
+    checkPushedInfo(df6,
+      "PushedAggregates: [SUM(SALARY)]",
+      "PushedGroupByExpressions: " +
+        "[CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY 
ELSE 0.00 END, " +
+        "IS_MANAGER]",
+      "PushedFilters: []",
+      "PushedTopN: ORDER BY [" +
+        "CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 
0.00 END " +
+        "ASC NULLS FIRST, IS_MANAGER ASC NULLS FIRST] LIMIT 1")
+    checkAnswer(df6, Seq(Row(0.00, false, 12000.00)))
+
+    val df7 = spark.read
+      .table("h2.test.employee")
+      .select($"DEPT", $"SALARY")
+      .groupBy("dept").agg(sum("SALARY"))
+      .orderBy(sum("SALARY"))

Review Comment:
   can we add some comments to explain why top n can't be pushed here?



-- 
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-unsubscr...@spark.apache.org

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