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 2021/12/24 06:15:53 UTC

[GitHub] [spark] beliefer opened a new pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

beliefer opened a new pull request #35009:
URL: https://github.com/apache/spark/pull/35009


   ### What changes were proposed in this pull request?
   Currently, Spark aggregate pushdown will translate some standard aggregate functions, so that compile these functions to adapt specify database.
   After this job, users could override `JdbcDialect.compileAggregate` to implement some aggregate functions supported by some database.
   Because some aggregate functions will be converted show below, this PR no need to match them.
   
   |Input|Parsed|Optimized|
   |------|--------------------|----------|
   |`Every`| `aggregate.BoolAnd` |`Min`|
   |`Any`| `aggregate.BoolOr` |`Max`|
   |`Some`| `aggregate.BoolOr` |`Max`|
   
   ### Why are the changes needed?
   Make the implement of `*Dialect` could extends the aggregate functions by override `JdbcDialect.compileAggregate`.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. Users could pushdown more aggregate functions.
   
   
   ### How was this patch tested?
   Exists tests.
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000731952


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/51028/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000814180


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/146553/
   


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


[GitHub] [spark] SparkQA commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000687357


   **[Test build #146553 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/146553/testReport)** for PR 35009 at commit [`b4e7e41`](https://github.com/apache/spark/commit/b4e7e41fe57469dcd8319cddfe2783ede3bc9e57).


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


[GitHub] [spark] SparkQA commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000705096


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/51028/
   


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


[GitHub] [spark] beliefer commented on a change in pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #35009:
URL: https://github.com/apache/spark/pull/35009#discussion_r775749479



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
##########
@@ -22,11 +22,35 @@ import java.util.Locale
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException}
+import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Average, StddevPop, StddevSamp, VarPop, VarSamp}
 
 private object H2Dialect extends JdbcDialect {
   override def canHandle(url: String): Boolean =
     url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2")
 
+  override def compileAggregate(aggFunction: AggregateFunc): Option[String] = {
+    super.compileAggregate(aggFunction).orElse(
+      aggFunction match {
+        case avg: Average =>
+          if (avg.column.fieldNames.length != 1) return None
+          Some(s"AVG(${quoteIdentifier(avg.column.fieldNames.head)})")

Review comment:
       OK




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


[GitHub] [spark] SparkQA commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000722980


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/51028/
   


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


[GitHub] [spark] beliefer closed pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
beliefer closed pull request #35009:
URL: https://github.com/apache/spark/pull/35009


   


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


[GitHub] [spark] SparkQA removed a comment on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000687357


   **[Test build #146553 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/146553/testReport)** for PR 35009 at commit [`b4e7e41`](https://github.com/apache/spark/commit/b4e7e41fe57469dcd8319cddfe2783ede3bc9e57).


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


[GitHub] [spark] beliefer commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
beliefer commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000724332


   ping @huaxingao cc @cloud-fan 


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000814180


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/146553/
   


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


[GitHub] [spark] beliefer commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
beliefer commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1013784225


   https://github.com/apache/spark/pull/35101 merged.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000731952


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/51028/
   


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


[GitHub] [spark] SparkQA commented on pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #35009:
URL: https://github.com/apache/spark/pull/35009#issuecomment-1000812012


   **[Test build #146553 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/146553/testReport)** for PR 35009 at commit [`b4e7e41`](https://github.com/apache/spark/commit/b4e7e41fe57469dcd8319cddfe2783ede3bc9e57).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `public final class Average implements AggregateFunc `
     * `public final class Corr implements AggregateFunc `
     * `public final class CovarPop implements AggregateFunc `
     * `public final class CovarSamp implements AggregateFunc `
     * `public final class StddevPop implements AggregateFunc `
     * `public final class StddevSamp implements AggregateFunc `
     * `public final class VarPop implements AggregateFunc `
     * `public final class VarSamp implements AggregateFunc `


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


[GitHub] [spark] huaxingao commented on a change in pull request #35009: [SPARK-37527][SQL] Translate more standard aggregate functions for pushdown

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #35009:
URL: https://github.com/apache/spark/pull/35009#discussion_r775712142



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
##########
@@ -22,11 +22,35 @@ import java.util.Locale
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException}
+import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Average, StddevPop, StddevSamp, VarPop, VarSamp}
 
 private object H2Dialect extends JdbcDialect {
   override def canHandle(url: String): Boolean =
     url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2")
 
+  override def compileAggregate(aggFunction: AggregateFunc): Option[String] = {
+    super.compileAggregate(aggFunction).orElse(
+      aggFunction match {
+        case avg: Average =>
+          if (avg.column.fieldNames.length != 1) return None
+          Some(s"AVG(${quoteIdentifier(avg.column.fieldNames.head)})")

Review comment:
       Should we put `avg` in `super.compileAggregate(aggFunction)` since all the databases support it?




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