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 2020/09/09 17:59:25 UTC

[GitHub] [spark] huaxingao opened a new pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

huaxingao opened a new pull request #29695:
URL: https://github.com/apache/spark/pull/29695


   
   ### What changes were proposed in this pull request?
   Push down JDBC aggregate to datasource layer
   
   ### Why are the changes needed?
   for better performance
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   add new 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.

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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #136320 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136320/testReport)** for PR 29695 at commit [`ef4bab9`](https://github.com/apache/spark/commit/ef4bab92827642c9e9ed5b45767e96e98debba82).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Count(column: String, dataType: DataType, isDistinct: Boolean) extends 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.

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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #135329 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135329/testReport)** for PR 29695 at commit [`69813c7`](https://github.com/apache/spark/commit/69813c7de38ca7e123e7f7dbb3367560ea12ce52).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #133905 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133905/testReport)** for PR 29695 at commit [`5921b7f`](https://github.com/apache/spark/commit/5921b7f5f26715c5ba9494a3cdc5dfd6f7ac4bdc).


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @rf972 Hi Robert, I have the fix in. Could you continue your TPC benchmark testing? Please let me know if there are any issues. 


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 closed pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   


-- 
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132961 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132961/testReport)** for PR 29695 at commit [`710ff80`](https://github.com/apache/spark/commit/710ff80aaad361bead755d78049cd56431cd65be).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,117 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
 object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression => agg
+            }
+          }.distinct
+
+          val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+            groupingExpressions)
+
+          val (pushedFilters, postScanFilters, scan, output, normalizedProjects) =
+            processFilerAndColumn(scanBuilder, project, filters, relation)
+
+          logInfo(
+            s"""
+               |Pushing operators to ${relation.name}
+               |Pushed Filters: ${pushedFilters.mkString(", ")}
+               |Post-Scan Filters: ${postScanFilters.mkString(",")}
+               |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+               |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+               |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+          val wrappedScan = scan match {
+            case v1: V1Scan =>
+              val translated = filters.flatMap(DataSourceStrategy.translateFilter(_, true))
+              V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+            case _ => scan
+          }
+
+          if (aggregation.aggregateExpressions.isEmpty) {
+            val plan = buildLogicalPlan(project, relation, wrappedScan, output, normalizedProjects,
+              postScanFilters)
+            Aggregate(groupingExpressions, resultExpressions, plan)
+          } else {
+            val resultAttributes = resultExpressions.map(_.toAttribute)
+              .map ( e => e match { case a: AttributeReference => a })
+            var index = 0
+            val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (a <- resultAttributes) {
+              aggOutputBuilder +=
+                a.copy(dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
+                  qualifier = a.qualifier)
+              index += 1
+            }
+            val aggOutput = aggOutputBuilder.result
+
+            var newOutput = aggOutput
+            for (col <- output) {
+              if (!aggOutput.exists(_.name.contains(col.name))) {
+                newOutput = col +: newOutput
+              }
+            }
+
+            val r = buildLogicalPlan(newOutput, relation, wrappedScan, newOutput,
+              normalizedProjects, postScanFilters)

Review comment:
       use the new output to build the plan. 
   e.g. original output, `JDBCScan$$anon$1@226de93c [ID#18]`, new output, `JDBCScan$$anon$1@3f6f9cef [max(ID)#77,min(ID)#78]`




----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   > For example, since PostgreSQL can have pretty large scale/precision in numeric, I think partial aggregated values can go over the value range of Spark decimals. So, I'm not sure that we can keep the same behaviour between with/without aggregate pushdown.
   
   If database supports larger precision and scale than Spark, I think we can adjust the precision and scale using either `DecimalType.bounded` or `DecimalType.adjustPrecisionScale`. 


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #135327 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135327/testReport)** for PR 29695 at commit [`e9e984e`](https://github.com/apache/spark/commit/e9e984ea0551d9c00df0ef5933dddcd7c3315c23).


----------------------------------------------------------------
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:
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] rf972 commented on pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   
   Thanks @huaxingao for the fix!  
   We were able to run the TPCH benchmark with your latest code and have these results for almost all of the TPCH tests.
   Although many of these tests do use aggregate, most cannot yet push down because of other portions of the query, which cannot be pushed down such as join and udf.  
   Our simulation shows that Q18 and Q20 will also benefit from aggregate pushdown, once Issue #1 is fixed.
   
   These test results were generated using our v2 data source https://github.com/rf972/s3datasource. And using our docker based spark/s3 test environment:  https://github.com/peterpuhov-github/dike
   
   ```
     TPCH                                                                  Aggregate
     Test    No Pushdown         Filter    Filter, Project    Aggregate    Difference
   ------  -------------  -------------  -----------------  -----------  ------------
        1    754,998,934    744,353,513        152,005,702  152,005,702             0
        2    288,283,676    240,552,680         19,640,050   19,640,050             0
        3    950,316,971    495,621,641         90,182,777   90,182,777             0
        4    925,953,149    761,524,915        179,910,653  179,910,653             0
        5    951,729,284    806,714,442        164,092,188  164,092,188             0
        6    754,998,934     14,320,106          1,565,944           17     1,565,927
        7    951,731,111    426,753,589         91,221,930   91,221,930             0
        8    975,686,104    833,050,619        211,038,608  211,038,608             0
        9  1,070,094,601  1,047,408,449        258,031,293  258,031,293             0
       10    950,319,184    216,925,396         55,738,379   55,738,379             0
       11    360,560,535    360,554,115         50,546,842   50,546,842             0
       12    925,953,149    203,531,165         37,713,185   37,713,185             0
       13    195,318,037    195,318,037         96,823,834   96,823,834             0
       14    778,953,541     33,509,818          7,152,453    7,152,453             0
       15  1,511,407,582     58,271,128          9,143,413     Issue #1
       16    144,139,239    143,184,520         17,410,512   17,410,512             0
       17  1,557,907,082  1,510,046,884        164,440,699  164,440,699             0
       18  1,705,315,905  1,705,315,905        183,515,611     Issue #1
       19    778,953,541     40,553,750          8,311,313    8,311,313             0
       20    899,140,386    258,559,476         34,420,567     Issue #1
       21  2,437,362,944  2,349,560,183        498,738,573  498,738,573             0
       22    219,681,859    217,455,483         17,168,762     Issue #1
   ```
   
   Issue #1 above is an exception we notice in Q15, Q18, Q20, and Q22.
   
   We came up with these tests to demonstrate the issue:
   ```
   val df = sparkSession.table("h2.test.employee")
   var query1 = df.select($"DEPT", $"SALARY".as("value"))
                  .groupBy($"DEPT")
                  .agg(sum($"value").as("total"))
                  .filter($"total" > 1000)
   query1.show()
   val decrease = udf { (x: Double, y: Double) => x - y}
   var query2 = df.select($"DEPT", decrease($"SALARY", $"BONUS").as("value"), $"SALARY", $"BONUS")
                  .groupBy($"DEPT")
                  .agg(sum($"value"), sum($"SALARY"), sum($"BONUS"))
   query2.show()
   ```
   Please let us know if additional details are needed on this.  Thanks !
   
   We are actively looking at using larger data sets to generate meaningful test timing.  
   We will share timing results as soon as we are done.
   
   


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132250 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132250/testReport)** for PR 29695 at commit [`f065399`](https://github.com/apache/spark/commit/f065399ac285a0d9af760a6b72baf3e664ecb4c2).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @rf972 
   I fixed the problem. 
   In your benchmark test, did you take the elapsed time for Q6? Is there any performance improvement?


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131097 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131097/testReport)** for PR 29695 at commit [`adb9986`](https://github.com/apache/spark/commit/adb9986125831af105c9fa021b3a70705c158696).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131179 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131179/testReport)** for PR 29695 at commit [`da0be52`](https://github.com/apache/spark/commit/da0be52b1fbab6368bfdd8c61cfabb71b20625b0).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #135327 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135327/testReport)** for PR 29695 at commit [`e9e984e`](https://github.com/apache/spark/commit/e9e984ea0551d9c00df0ef5933dddcd7c3315c23).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   cc @cloud-fan @maropu 


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #133905 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133905/testReport)** for PR 29695 at commit [`5921b7f`](https://github.com/apache/spark/commit/5921b7f5f26715c5ba9494a3cdc5dfd6f7ac4bdc).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130238 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130238/testReport)** for PR 29695 at commit [`bafdba4`](https://github.com/apache/spark/commit/bafdba4d0a932417cac1ba4975549a8d9f5423fa).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130240/testReport)** for PR 29695 at commit [`5e92c25`](https://github.com/apache/spark/commit/5e92c25661df1245f3c75eac83a71eb75918665b).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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] rf972 commented on pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   @huaxingao Thanks for merging aggregate push down !  We have tested this Spark 3.2 aggregate push down support with [our own push down capable datasource for S3 and HDFS](https://github.com/open-infrastructure-labs/caerus-pushdown-datasource).  We have run all our tests and we have not found any issues.


-- 
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132940 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132940/testReport)** for PR 29695 at commit [`710ff80`](https://github.com/apache/spark/commit/710ff80aaad361bead755d78049cd56431cd65be).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #133905 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/133905/testReport)** for PR 29695 at commit [`5921b7f`](https://github.com/apache/spark/commit/5921b7f5f26715c5ba9494a3cdc5dfd6f7ac4bdc).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131444 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131444/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
##########
@@ -264,6 +265,19 @@ private[sql] case class JDBCRelation(
     }
   }
 
+  override def unhandledAggregates(aggregates: Array[AggregateFunc]):
+    Array[AggregateFunc] = {
+    if (jdbcOptions.pushDownAggregate) {
+      if (JDBCRDD.compileAggregates(aggregates, JdbcDialects.get(jdbcOptions.url)).isEmpty) {
+        aggregates
+      } else {
+        Array.empty[AggregateFunc]
+      }
+    } else {
+      aggregates
+    }
+  }

Review comment:
       will remove this method

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,117 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
 object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression => agg
+            }
+          }.distinct
+
+          val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+            groupingExpressions)
+
+          val (pushedFilters, postScanFilters, scan, output, normalizedProjects) =
+            processFilerAndColumn(scanBuilder, project, filters, relation)
+
+          logInfo(
+            s"""
+               |Pushing operators to ${relation.name}
+               |Pushed Filters: ${pushedFilters.mkString(", ")}
+               |Post-Scan Filters: ${postScanFilters.mkString(",")}
+               |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+               |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+               |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+          val wrappedScan = scan match {
+            case v1: V1Scan =>
+              val translated = filters.flatMap(DataSourceStrategy.translateFilter(_, true))
+              V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+            case _ => scan
+          }
+
+          if (aggregation.aggregateExpressions.isEmpty) {
+            val plan = buildLogicalPlan(project, relation, wrappedScan, output, normalizedProjects,
+              postScanFilters)
+            Aggregate(groupingExpressions, resultExpressions, plan)
+          } else {
+            val resultAttributes = resultExpressions.map(_.toAttribute)
+              .map ( e => e match { case a: AttributeReference => a })
+            var index = 0
+            val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (a <- resultAttributes) {
+              aggOutputBuilder +=
+                a.copy(dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
+                  qualifier = a.qualifier)
+              index += 1
+            }
+            val aggOutput = aggOutputBuilder.result
+
+            var newOutput = aggOutput
+            for (col <- output) {
+              if (!aggOutput.exists(_.name.contains(col.name))) {
+                newOutput = col +: newOutput
+              }
+            }
+
+            val r = buildLogicalPlan(newOutput, relation, wrappedScan, newOutput,
+              normalizedProjects, postScanFilters)

Review comment:
       use the new output to build the plan. 
   e.g. original output, `JDBCScan$$anon$1@226de93c [ID#18]`, new output, `JDBCScan$$anon$1@3f6f9cef [max(ID)#77,min(ID)#78]`

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,117 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
 object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression => agg
+            }
+          }.distinct
+
+          val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+            groupingExpressions)
+
+          val (pushedFilters, postScanFilters, scan, output, normalizedProjects) =
+            processFilerAndColumn(scanBuilder, project, filters, relation)
+
+          logInfo(
+            s"""
+               |Pushing operators to ${relation.name}
+               |Pushed Filters: ${pushedFilters.mkString(", ")}
+               |Post-Scan Filters: ${postScanFilters.mkString(",")}
+               |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+               |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+               |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+          val wrappedScan = scan match {
+            case v1: V1Scan =>
+              val translated = filters.flatMap(DataSourceStrategy.translateFilter(_, true))
+              V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+            case _ => scan
+          }
+
+          if (aggregation.aggregateExpressions.isEmpty) {
+            val plan = buildLogicalPlan(project, relation, wrappedScan, output, normalizedProjects,
+              postScanFilters)
+            Aggregate(groupingExpressions, resultExpressions, plan)
+          } else {
+            val resultAttributes = resultExpressions.map(_.toAttribute)
+              .map ( e => e match { case a: AttributeReference => a })
+            var index = 0
+            val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (a <- resultAttributes) {
+              aggOutputBuilder +=
+                a.copy(dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
+                  qualifier = a.qualifier)
+              index += 1
+            }
+            val aggOutput = aggOutputBuilder.result
+
+            var newOutput = aggOutput
+            for (col <- output) {
+              if (!aggOutput.exists(_.name.contains(col.name))) {
+                newOutput = col +: newOutput
+              }
+            }
+
+            val r = buildLogicalPlan(newOutput, relation, wrappedScan, newOutput,
+              normalizedProjects, postScanFilters)
+            val plan = Aggregate(groupingExpressions, resultExpressions, r)
+
+            var i = 0
+            plan.transformExpressions {
+            case agg: AggregateExpression =>
+              val aggFunction: aggregate.AggregateFunction = {
+                i += 1
+                if (agg.aggregateFunction.isInstanceOf[aggregate.Max]) {
+                  aggregate.Max(aggOutput(i - 1))
+                } else if (agg.aggregateFunction.isInstanceOf[aggregate.Min]) {
+                  aggregate.Min(aggOutput(i - 1))
+                } else if (agg.aggregateFunction.isInstanceOf[aggregate.Average]) {
+                  aggregate.Average(aggOutput(i - 1))
+                } else if (agg.aggregateFunction.isInstanceOf[aggregate.Sum]) {
+                  aggregate.Sum(aggOutput(i - 1))
+                } else {
+                  agg.aggregateFunction
+                }
+              }
+              agg.transform {
+                case a: aggregate.AggregateFunction => aggFunction
+              }

Review comment:
       update optimized logical plan 
   before update
   ```
   Aggregate [max(id#18) AS max(id)#21, min(id#18) AS min(id)#22]
   +- RelationV2[ID#18] test.people
   ```
   after update
   ```
   Aggregate [max(max(ID)#77) AS max(ID)#72, min(min(ID)#78) AS min(ID)#73]
   +- RelationV2[max(ID)#77, min(ID)#78] test.people
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
##########
@@ -232,6 +232,17 @@ abstract class BaseRelation {
    * @since 1.6.0
    */
   def unhandledFilters(filters: Array[Filter]): Array[Filter] = filters
+
+  /**
+   * Returns the list of [[Aggregate]]s that this datasource may not be able to handle.
+   * These returned [[Aggregate]]s will be evaluated by Spark SQL after data is output by a scan.
+   * By default, this function will return all aggregates, as it is always safe to
+   * double evaluate a [[Aggregate]].
+   *
+   * @since 3.1.0
+   */
+  def unhandledAggregates(aggregates: Array[AggregateFunc]): Array[AggregateFunc] =
+    aggregates

Review comment:
       This is not used. Will remove




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   **[Test build #136600 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136600/testReport)** for PR 29695 at commit [`782a0a8`](https://github.com/apache/spark/commit/782a0a827fbb1e60b8673d533cbda618df65dc96).


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132250 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132250/testReport)** for PR 29695 at commit [`f065399`](https://github.com/apache/spark/commit/f065399ac285a0d9af760a6b72baf3e664ecb4c2).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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] maropu commented on pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   > Can we support aggregate pushdown with all the data types? For example, aggregating decimal values seems to have different behaviours between database implementations.
   >> Sorry I am not familiar with this. Could you please give me an example of the different implementations?
   
   For example, since PostgreSQL can have pretty large scale/precision in numeric, I think partial aggregated values can go over the value range of Spark decimals. So, I'm not sure that we can keep the same behaviour between with/without aggregate pushdown.
   
   


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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] maropu commented on a change in pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       What if `SALARY` is a tiny int? Looks `sum(SALARY)` can have an overflow easily in the datasource side.




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128925 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128925/testReport)** for PR 29695 at commit [`adf1588`](https://github.com/apache/spark/commit/adf1588218a94e88fda7b7a90e824215b363cfcf).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Aggregation(aggregateExpressions: Seq[AggregateFunc],`
     * `case class Avg(column: String) extends AggregateFunc`
     * `case class Min(column: String) extends AggregateFunc`
     * `case class Max(column: String) extends AggregateFunc`
     * `case class Sum(column: String) extends 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.

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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   also cc @sunchao @dongjoon-hyun 


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128536 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128536/testReport)** for PR 29695 at commit [`78bc331`](https://github.com/apache/spark/commit/78bc331ab7c0143922bc6552fe565eac7a498454).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132961 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132961/testReport)** for PR 29695 at commit [`710ff80`](https://github.com/apache/spark/commit/710ff80aaad361bead755d78049cd56431cd65be).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128673 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128673/testReport)** for PR 29695 at commit [`c45a2b6`](https://github.com/apache/spark/commit/c45a2b643f2e83af14f6b0584c5c76cf1e5af4c0).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130239/testReport)** for PR 29695 at commit [`5e92c25`](https://github.com/apache/spark/commit/5e92c25661df1245f3c75eac83a71eb75918665b).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @cloud-fan Could you take a look, please? It is still WIP and many places need to be improved, but it would be very helpful if you could please take a quick look to see if the approach is okay. Thank you very much!


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128463 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128463/testReport)** for PR 29695 at commit [`8ccefd9`](https://github.com/apache/spark/commit/8ccefd989331228d372e1c43ca9303f3ce79d11b).


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @maropu Thank you very much for your review. 
   
   > Can we support aggregate pushdown with all the data types? For example, aggregating decimal values seems to have different behaviours between database implementations.
   
   Sorry I am not familiar with this. Could you please give me an example of the different implementations?
   
   > How does Spark receive aggregated values on database sides? It seems the data types of input/aggregated values are different in some databases, e.g., sum(bigint)=>numeric in PostgreSQL.
   
   I will cast the output of aggregates to the type that spark expects.
   For example, spark expects bigInt from sum(int), so the output of sum(int) from database needs to be casted to bigint.
   
   
   > How does Spark handle overflows on database sides?
   
   Not sure how to handle this yet. I will try to figure out. Please let me know if you have a good idea. 
   
   


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128461 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128461/testReport)** for PR 29695 at commit [`8257314`](https://github.com/apache/spark/commit/82573149c12dee977d34d467cf87b1926ad20673).


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   @moderakh Thanks for the interest in this PR. We are working with @cloud-fan on this Aggregate push down feature. Hopefully we can reach a consensus soon and are able to move forward.


-- 
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:
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] moderakh edited a comment on pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

Posted by GitBox <gi...@apache.org>.
moderakh edited a comment on pull request #29695:
URL: https://github.com/apache/spark/pull/29695#issuecomment-832239822


   we are building a spark connector (https://github.com/Azure/azure-sdk-for-java/tree/master/sdk/cosmos/azure-cosmos-spark_3-1_2-12) using DataSoruceV2 api and are interested in aggregate push-down support, and we are excited about seeing your PR. :-)
   
   @huaxingao @sunchao @viirya I wonder what the status of aggregate push-down for spark3 is. Are you planning to include this change in spark 3.2?


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128664 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128664/testReport)** for PR 29695 at commit [`c45a2b6`](https://github.com/apache/spark/commit/c45a2b643f2e83af14f6b0584c5c76cf1e5af4c0).


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   cc @viirya 


----------------------------------------------------------------
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:
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] viirya commented on a change in pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownAggregates.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.connector.read;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.AggregateFunction;
+
+/**
+ * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
+ * push down aggregates to the data source.
+ *
+ * @since 3.1.0
+ */
+@Evolving
+public interface SupportsPushDownAggregates extends ScanBuilder {
+
+  /**
+   * Pushes down Aggregation and returns aggregates that need to be evaluated after scanning.
+   * The Aggregation can be pushed down only if all the Aggregate Functions can
+   * be pushed down.
+   * @since 3.1.0

Review comment:
       do we need since for the methods here? `pushFilters` and `pushedFilters` don't have it.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -133,6 +133,23 @@ object JDBCRDD extends Logging {
     })
   }
 
+  def compileAggregates(aggregates: Seq[AggregateFunction], dialect: JdbcDialect):
+    Map[String, String] = {
+    def quote(colName: String): String = dialect.quoteIdentifier(colName)
+    val compiledAggregates = aggregates.map {
+      case Min(column) => Some(quote(column) -> s"MIN(${quote(column)})")
+      case Max(column) => Some(quote(column) -> s"MAX(${quote(column)})")
+      case Sum(column) => Some(quote(column) -> s"Sum(${quote(column)})")
+      case Avg(column) => Some(quote(column) -> s"Avg(${quote(column)})")

Review comment:
       Why MIN and MAX are all upper-case, Sum and Avg are not?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -133,6 +133,23 @@ object JDBCRDD extends Logging {
     })
   }
 
+  def compileAggregates(aggregates: Seq[AggregateFunction], dialect: JdbcDialect):
+    Map[String, String] = {
+    def quote(colName: String): String = dialect.quoteIdentifier(colName)
+    val compiledAggregates = aggregates.map {
+      case Min(column) => Some(quote(column) -> s"MIN(${quote(column)})")
+      case Max(column) => Some(quote(column) -> s"MAX(${quote(column)})")
+      case Sum(column) => Some(quote(column) -> s"Sum(${quote(column)})")
+      case Avg(column) => Some(quote(column) -> s"Avg(${quote(column)})")
+      case _ => None
+    }
+    if (!compiledAggregates.contains(None)) {
+      compiledAggregates.flatMap(x => x).toMap

Review comment:
       Since `column` is key, each column can have only one aggregate function?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/aggregates.scala
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.sources
+
+case class Aggregation(aggregateExpressions: Seq[AggregateFunction],
+                       groupByExpressions: Seq[String])
+
+abstract class AggregateFunction
+
+case class Avg (column: String) extends AggregateFunction
+
+case class Min (column: String) extends AggregateFunction
+
+case class Max (column: String) extends AggregateFunction
+
+case class Sum (column: String) extends AggregateFunction

Review comment:
       nit: remove extra space like `Avg(column: String)`

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala
##########
@@ -69,6 +71,37 @@ object PushDownUtils extends PredicateHelper {
     }
   }
 
+    /**
+     * Pushes down aggregates to the data source reader
+     *
+     * @return pushed aggregates and post-scan aggregates.
+     */
+    def pushAggregates(scanBuilder: ScanBuilder, aggregates: Seq[AggregateExpression])
+      : (Seq[sources.AggregateFunction], Seq[AggregateExpression]) = {
+      scanBuilder match {
+        case r: SupportsPushDownAggregates =>
+          val translatedAggregates = mutable.ArrayBuffer.empty[sources.AggregateFunction]
+          // Catalyst aggregate expression that can't be translated to data source aggregates.
+          val untranslatableExprs = mutable.ArrayBuffer.empty[AggregateExpression]
+
+          for (aggregateExpr <- aggregates) {
+            val translated = DataSourceStrategy.translateAggregate(aggregateExpr)
+            if (translated.isEmpty) {
+              untranslatableExprs += aggregateExpr
+            } else {
+              translatedAggregates += translated.get
+            }
+          }
+
+          if (untranslatableExprs.isEmpty) r.pushAggregates(translatedAggregates.toArray)
+
+          // push down only if all the aggregates can be pushed down
+          if (!r.pushedAggregates.isEmpty) (r.pushedAggregates, Nil) else (Nil, aggregates)

Review comment:
       Even we can pushdown aggregate functions, we don't need post-scan aggregates at all?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -640,6 +647,21 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunction] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+    }
+
+    aggregates.aggregateFunction match {
+      case aggregate.Min(child) => Some(Min(columnAsString(child)))

Review comment:
       If `child` is not a `AttributeReference`, can this work?




----------------------------------------------------------------
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:
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] baibaichen commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuilder
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughNonJoin.{getAliasMap, replaceAlias}

Review comment:
       I think `V2ScanRelationPushDown` should extend with `AliasHelper` to avoid change access qualifier of getAliasMap




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131400 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131400/testReport)** for PR 29695 at commit [`7de8b20`](https://github.com/apache/spark/commit/7de8b202525bdf7b774a030da1e180ccabf66ed0).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131430 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131430/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   **[Test build #136600 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136600/testReport)** for PR 29695 at commit [`782a0a8`](https://github.com/apache/spark/commit/782a0a827fbb1e60b8673d533cbda618df65dc96).


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128936 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128936/testReport)** for PR 29695 at commit [`adf1588`](https://github.com/apache/spark/commit/adf1588218a94e88fda7b7a90e824215b363cfcf).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Aggregation(aggregateExpressions: Seq[AggregateFunc],`
     * `case class Avg(column: String) extends AggregateFunc`
     * `case class Min(column: String) extends AggregateFunc`
     * `case class Max(column: String) extends AggregateFunc`
     * `case class Sum(column: String) extends 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.

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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131097 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131097/testReport)** for PR 29695 at commit [`adb9986`](https://github.com/apache/spark/commit/adb9986125831af105c9fa021b3a70705c158696).


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128588 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128588/testReport)** for PR 29695 at commit [`b11b24b`](https://github.com/apache/spark/commit/b11b24bc29e8759fced0efe16568701e0eb30d42).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128936 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128936/testReport)** for PR 29695 at commit [`adf1588`](https://github.com/apache/spark/commit/adf1588218a94e88fda7b7a90e824215b363cfcf).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128936 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128936/testReport)** for PR 29695 at commit [`adf1588`](https://github.com/apache/spark/commit/adf1588218a94e88fda7b7a90e824215b363cfcf).


----------------------------------------------------------------
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:
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] baibaichen commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuilder
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughNonJoin.{getAliasMap, replaceAlias}

Review comment:
       I think V2ScanRelationPushDown should extend with AliasHelper to avoid change access qualifier of getAliasMap




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132912 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132912/testReport)** for PR 29695 at commit [`b508663`](https://github.com/apache/spark/commit/b50866377cc5ef8c6bc9857ef8a1e893f300931c).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Build finished. Test FAILed.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128463 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128463/testReport)** for PR 29695 at commit [`8ccefd9`](https://github.com/apache/spark/commit/8ccefd989331228d372e1c43ca9303f3ce79d11b).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @rf972 Thanks for helping with the performance test. I will continue working on this PR and hopefully it can get in.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132250 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132250/testReport)** for PR 29695 at commit [`f065399`](https://github.com/apache/spark/commit/f065399ac285a0d9af760a6b72baf3e664ecb4c2).


----------------------------------------------------------------
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:
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] baibaichen commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       @maropu @huaxingao 
   
   such casting are removed by spark in https://github.com/apache/spark/pull/31079




-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132968 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132968/testReport)** for PR 29695 at commit [`710ff80`](https://github.com/apache/spark/commit/710ff80aaad361bead755d78049cd56431cd65be).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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] moderakh commented on pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   we are building a spark connector using DataSoruceV2 api and are interested in aggregate push-down support.
   https://github.com/Azure/azure-sdk-for-java/tree/master/sdk/cosmos/azure-cosmos-spark_3-1_2-12
   
   I wonder what the status of aggregate push-down for spark3 is. Are you planning to include this change in spark 3.2?
   
   @huaxingao @sunchao @viirya


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #135327 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135327/testReport)** for PR 29695 at commit [`e9e984e`](https://github.com/apache/spark/commit/e9e984ea0551d9c00df0ef5933dddcd7c3315c23).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Avg(column: String, dataType: DataType, isDistinct: Boolean) extends AggregateFunc`
     * `case class Min(column: String, dataType: DataType) extends AggregateFunc`
     * `case class Max(column: String, dataType: DataType) extends AggregateFunc`
     * `case class Sum(column: String, dataType: DataType, isDistinct: Boolean) extends 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.

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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128461 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128461/testReport)** for PR 29695 at commit [`8257314`](https://github.com/apache/spark/commit/82573149c12dee977d34d467cf87b1926ad20673).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Avg (column: String) extends Aggregate`
     * `case class Min (column: String) extends Aggregate`
     * `case class Max (column: String) extends Aggregate`
     * `case class Sum (column: String) extends Aggregate`
     * `trait PrunedFilteredAggregateScan `


----------------------------------------------------------------
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:
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] rf972 commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -73,33 +77,25 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] {
               postScanFilters)
             Aggregate(groupingExpressions, resultExpressions, plan)
           } else {
-            val resultAttributes = resultExpressions.map(_.toAttribute)
-              .map ( e => e match { case a: AttributeReference => a })
-            var index = 0
             val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
-            for (a <- resultAttributes) {
-              val newName = if (a.name.contains("FILTER")) {
-                a.name.substring(0, a.name.indexOf("FILTER") - 1)
-              } else if (a.name.contains("DISTINCT")) {
-                a.name.replace("DISTINCT ", "")
-              } else {
-                a.name
-              }
-
-              aggOutputBuilder +=
-                a.copy(name = newName,
-                  dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
-                  qualifier = a.qualifier)
-              index += 1
+            for (a <- aggregates) {
+                aggOutputBuilder += AttributeReference(toPrettySQL(a), a.dataType)()
             }
             val aggOutput = aggOutputBuilder.result
 
-            var newOutput = aggOutput
-            for (col <- output) {
-              if (!aggOutput.exists(_.name.contains(col.name))) {
-                newOutput = col +: newOutput
+            val newOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (col1 <- output) {
+              var found = false
+              for (col2 <- aggOutput) {
+                  if (contains(col2.name, col1.name)) {

Review comment:
       Thanks @huaxingao  for incorporating our suggestions into the patch regarding aggregates containing expressions!  
   With your changes, TPCH Q6 test shows a substantial reduction in data transfer.  
   With just filter and projection pushdown, Q6 transfer size is about 1.5 MB.  But with aggregate pushdown this now gets reduced to 17 bytes.
   
   In our testing we found a case that throws an error.  Grouping by more than one column seems to cause an error.  Here is an example that fails for us:
   
   ```
   val df = sparkSession.sql("select BONUS, SUM(SALARY+BONUS), SALARY FROM h2.test.employee" + 
          " GROUP BY SALARY, BONUS")
   df.show()
   ```
   
   If you have any thoughts or suggestions on a solution for this, we would appreciate it.  Thanks !
   




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128461 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128461/testReport)** for PR 29695 at commit [`8257314`](https://github.com/apache/spark/commit/82573149c12dee977d34d467cf87b1926ad20673).


----------------------------------------------------------------
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:
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] rf972 commented on pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Thanks @huaxingao for the fixes !  We are making it further in our testing, but did find a few issues running tpch with the latest patch.  We did our best here to translate the failing cases into examples that fit into JDBCV2Suite.scala.
   
   We noticed an issue in the case where only some of the filters are pushed, but yet the aggregate operation is still being pushed down.  This results in an exception, and we believe that we should only push down aggregates if all of the filters can be pushed down.  Here is an example test with a filter containing a UDF, which cannot be pushed down:
   
   ```
   val df1 = sparkSession.table("h2.test.employee")
   val sub2 = udf { (x: String) => x.substring(0, 3) }
   val name = udf { (x: String) => x.matches("cat|dav|amy") }
   val df2 = df1.select($"SALARY", $"BONUS", sub2($"NAME").as("nsub2"))
                .filter("SALARY > 100")
                .filter(name($"nsub2"))
                .agg(avg($"SALARY").as("avg_salary"))
   df2.explain(true)
   df2.show()
   ```
   
   Another issue we found is seen by the below example.
   ```
   val df1 = sparkSession.table("h2.test.employee")
   df1.filter($"DEPT" > 0 && $"SALARY" >= 0.05).agg(sum($"BONUS" * $"SALARY")).show()
   ```
   
   We noticed that upper case seems to be needed for aggregates.  So while these tests pass,
   ```
   df1.filter($"dept" > 0 && $"salary" > 9000).show()
   df1.filter($"dept" > 0 && $"salary" > 9000).agg(sum($"SALARY")).show()
   ```
   
   other tests like the below fail with lower case in the aggregate.
   `df1.filter($"dept" > 0 && $"salary" > 9000).agg(sum($"salary")).show()`
   
   This case issue is admittedly a nit, but we saw it in our testing, so we decided to bring it up.
   
   As always please let us know if more details are needed.  Thanks !


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128925 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128925/testReport)** for PR 29695 at commit [`adf1588`](https://github.com/apache/spark/commit/adf1588218a94e88fda7b7a90e824215b363cfcf).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128466 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128466/testReport)** for PR 29695 at commit [`f88e896`](https://github.com/apache/spark/commit/f88e896315f0e8574a48f2f9b1453de85d456566).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   also cc @cloud-fan @maropu @MaxGekk 
   I did some initial work for aggregate push down. There are still quite some places to be improved, but could you please take a look to see if this is OK? Thanks!


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128663 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128663/testReport)** for PR 29695 at commit [`134a83e`](https://github.com/apache/spark/commit/134a83e59df79bcee8a9df53ad5584ae2fbe4d17).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128466 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128466/testReport)** for PR 29695 at commit [`f88e896`](https://github.com/apache/spark/commit/f88e896315f0e8574a48f2f9b1453de85d456566).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128664 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128664/testReport)** for PR 29695 at commit [`c45a2b6`](https://github.com/apache/spark/commit/c45a2b643f2e83af14f6b0584c5c76cf1e5af4c0).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @viirya Thanks for reviewing. I have addressed your comments. Could you please check one more time?


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   @rf972 Thank you very much for taking time evaluating this patch! Also thanks for the reproduce. I know there are some issues. I am busy with something else recently and probably don't have much time to work on this in a week or two.  But I will definitely come back to this and fix the problems. 


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128673 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128673/testReport)** for PR 29695 at commit [`c45a2b6`](https://github.com/apache/spark/commit/c45a2b643f2e83af14f6b0584c5c76cf1e5af4c0).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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] maropu commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
##########
@@ -191,6 +191,9 @@ class JDBCOptions(
   // An option to allow/disallow pushing down predicate into JDBC data source
   val pushDownPredicate = parameters.getOrElse(JDBC_PUSHDOWN_PREDICATE, "true").toBoolean
 
+  // An option to allow/disallow pushing down aggregate into JDBC data source
+  val pushDownAggregate = parameters.getOrElse(JDBC_PUSHDOWN_AGGREGATE, "false").toBoolean

Review comment:
       I think this should be documented in https://spark.apache.org/docs/latest/sql-data-sources-jdbc.html

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,51 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {
+    case AttributeReference(name, _, _, _) => name
+    case Cast(child, _, _) => columnAsString (child)
+    case Add(left, right, _) =>
+      columnAsString(left) + " + " + columnAsString(right)
+    case Subtract(left, right, _) =>
+      columnAsString(left) + " - " + columnAsString(right)
+    case Multiply(left, right, _) =>
+      columnAsString(left) + " * " + columnAsString(right)
+    case Divide(left, right, _) =>
+      columnAsString(left) + " / " + columnAsString(right)
+    case CheckOverflow(child, _, _) => columnAsString (child)
+    case PromotePrecision(child) => columnAsString (child)
+    case _ => ""
+  }
+
+  protected[sql] def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+

Review comment:
       nit: unnecessary blank.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,41 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {

Review comment:
       > This covers a lot of cases but also makes it easy to break. We can begin with simplest case and add more supports later.
   
   +1
   
   

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -181,21 +248,53 @@ private[jdbc] class JDBCRDD(
     filters: Array[Filter],
     partitions: Array[Partition],
     url: String,
-    options: JDBCOptions)
+    options: JDBCOptions,
+    aggregation: Aggregation = Aggregation.empty)
   extends RDD[InternalRow](sc, Nil) {
 
   /**
    * Retrieve the list of partitions corresponding to this RDD.
    */
   override def getPartitions: Array[Partition] = partitions
 
+  private var updatedSchema: StructType = new StructType()

Review comment:
       Could we avoid to use `var` here?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala
##########
@@ -77,15 +77,25 @@ case class Count(children: Seq[Expression]) extends DeclarativeAggregate {
 
   override def defaultResult: Option[Literal] = Option(Literal(0L))
 
+  private[sql] var pushDown: Boolean = false

Review comment:
       I feel putting this variable for the pushdown feature does not look a good design. We cannot extend `Count` for pushdown-specific counting?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,51 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {
+    case AttributeReference(name, _, _, _) => name
+    case Cast(child, _, _) => columnAsString (child)
+    case Add(left, right, _) =>
+      columnAsString(left) + " + " + columnAsString(right)
+    case Subtract(left, right, _) =>
+      columnAsString(left) + " - " + columnAsString(right)
+    case Multiply(left, right, _) =>
+      columnAsString(left) + " * " + columnAsString(right)
+    case Divide(left, right, _) =>
+      columnAsString(left) + " / " + columnAsString(right)
+    case CheckOverflow(child, _, _) => columnAsString (child)
+    case PromotePrecision(child) => columnAsString (child)
+    case _ => ""
+  }
+
+  protected[sql] def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    aggregates.aggregateFunction match {
+      case min: aggregate.Min =>
+        val colName = columnAsString(min.child)
+        if (colName.nonEmpty) Some(Min(colName, min.dataType)) else None
+      case max: aggregate.Max =>
+        val colName = columnAsString(max.child)
+        if (colName.nonEmpty) Some(Max(colName, max.dataType)) else None
+      case avg: aggregate.Average =>
+        val colName = columnAsString(avg.child)
+        if (colName.nonEmpty) Some(Avg(colName, avg.dataType, aggregates.isDistinct)) else None
+      case sum: aggregate.Sum =>
+        val colName = columnAsString(sum.child)
+        if (colName.nonEmpty) Some(Sum(colName, sum.dataType, aggregates.isDistinct)) else None
+      case count: aggregate.Count =>
+        val columnName = count.children.head match {
+          case Literal(_, _) =>
+            "1"

Review comment:
       nit: `          case Literal(_, _) => "1"`




-- 
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   JDBC aggregate push down is merged in 3.2 using this PR https://github.com/apache/spark/pull/33352. There are several limitations: 
   1. Avg is not supported yet. I will probably work on this in 3.3
   2. aggregate over alias is not pushed down
   3. aggregate over arithmetic operation is not pushed down
   
   @rf972 If you have time, please try it out and let me know if there are any issues. Thanks a lot!
   
   I will close this PR. Thank you everyone for helping me reviewing and testing! Really appreciate all your help!


-- 
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] moderakh edited a comment on pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

Posted by GitBox <gi...@apache.org>.
moderakh edited a comment on pull request #29695:
URL: https://github.com/apache/spark/pull/29695#issuecomment-832239822


   we are developing a spark connector (https://github.com/Azure/azure-sdk-for-java/tree/master/sdk/cosmos/azure-cosmos-spark_3-1_2-12) using DataSoruceV2 api and are interested in aggregate push-down support. 
   
   We are excited about seeing your PR. :-)
   
   @huaxingao @sunchao @viirya I wonder what the status of aggregate push-down for spark3 is. Are you planning to include this change in spark 3.2?


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132968 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132968/testReport)** for PR 29695 at commit [`710ff80`](https://github.com/apache/spark/commit/710ff80aaad361bead755d78049cd56431cd65be).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       Thanks for the example. I actually only want to strip off the cast added by Spark. For example, when doing sum, Spark cast integral type to long. 
   ```
   case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType))
   ```
   For the casting added by Spark, I will remove the casting, push down aggregate and do the same casting on database side.
   If the cast is from user, I will keep the cast and NOT push down `aggregate (cast(col))` for now.
   To differentiate user explicitly casting from Spark added casting, I will add a flag some where. Does this sound OK to you?
   
   




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
##########
@@ -232,6 +232,17 @@ abstract class BaseRelation {
    * @since 1.6.0
    */
   def unhandledFilters(filters: Array[Filter]): Array[Filter] = filters
+
+  /**
+   * Returns the list of [[Aggregate]]s that this datasource may not be able to handle.
+   * These returned [[Aggregate]]s will be evaluated by Spark SQL after data is output by a scan.
+   * By default, this function will return all aggregates, as it is always safe to
+   * double evaluate a [[Aggregate]].
+   *
+   * @since 3.1.0
+   */
+  def unhandledAggregates(aggregates: Array[AggregateFunc]): Array[AggregateFunc] =
+    aggregates

Review comment:
       This is not used. Will remove




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130239 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130239/testReport)** for PR 29695 at commit [`5e92c25`](https://github.com/apache/spark/commit/5e92c25661df1245f3c75eac83a71eb75918665b).


----------------------------------------------------------------
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:
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] sunchao commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,49 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {
+    case AttributeReference(name, _, _, _) => name
+    case Cast(child, _, _) => columnAsString (child)
+    case Add(left, right, _) =>
+      columnAsString(left) + " + " + columnAsString(right)
+    case Subtract(left, right, _) =>
+      columnAsString(left) + " - " + columnAsString(right)
+    case Multiply(left, right, _) =>
+      columnAsString(left) + " * " + columnAsString(right)
+    case Divide(left, right, _) =>
+      columnAsString(left) + " / " + columnAsString(right)
+    case CheckOverflow(child, _, _) => columnAsString (child)

Review comment:
       nit: extra space after `columnAsString`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,132 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>

Review comment:
       This is a little hard to read. Maybe we can better separate the logic for pushing down aggregate with pushing down filters. Also some comments can help.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,132 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
-      val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
-        normalizedFilters.partition(SubqueryExpression.hasSubquery)
+          val aliasMap = getAliasMap(project)
+          var aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression =>
+                replaceAlias(agg, aliasMap).asInstanceOf[AggregateExpression]
+            }
+          }
+          aggregates = DataSourceStrategy.normalizeExprs(aggregates, relation.output)
+            .asInstanceOf[Seq[AggregateExpression]]
 
-      // `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)
-      val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery
+          val groupingExpressionsWithoutAlias = groupingExpressions.flatMap{ expr =>
+            expr.collect {
+              case e: Expression => replaceAlias(e, aliasMap)
+            }
+          }
+          val normalizedGroupingExpressions =
+            DataSourceStrategy.normalizeExprs(groupingExpressionsWithoutAlias, relation.output)
+
+          var newFilters = filters
+          aggregates.foreach(agg =>
+            if (agg.filter.nonEmpty)  {
+              // handle agg filter the same way as other filters
+              newFilters = newFilters :+ agg.filter.get
+            }
+          )
+
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, newFilters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)

Review comment:
       perhaps we should return the original plan node rather than a new `Aggregate`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -133,6 +135,68 @@ object JDBCRDD extends Logging {
     })
   }
 
+  private def containsArithmeticOp(col: String): Boolean =
+    col.contains("+") || col.contains("-") || col.contains("*") || col.contains("/")
+
+  def compileAggregates(
+      aggregates: Seq[AggregateFunc],
+      dialect: JdbcDialect): (Array[String], Array[DataType]) = {
+    def quote(colName: String): String = dialect.quoteIdentifier(colName)
+    val aggBuilder = ArrayBuilder.make[String]
+    val dataTypeBuilder = ArrayBuilder.make[DataType]
+    aggregates.map {
+      case Min(column, dataType) =>
+        dataTypeBuilder += dataType
+        if (!containsArithmeticOp(column)) {
+          aggBuilder += s"MIN(${quote(column)})"
+        } else {
+          aggBuilder += s"MIN(${quoteEachCols(column, dialect)})"
+        }
+      case Max(column, dataType) =>
+        dataTypeBuilder += dataType
+        if (!containsArithmeticOp(column)) {
+          aggBuilder += s"MAX(${quote(column)})"
+        } else {
+          aggBuilder += s"MAX(${quoteEachCols(column, dialect)})"
+        }
+      case Sum(column, dataType, isDistinct) =>
+        val distinct = if (isDistinct) "DISTINCT " else ""
+        dataTypeBuilder += dataType
+        if (!containsArithmeticOp(column)) {
+          aggBuilder += s"SUM(${distinct} ${quote(column)})"
+        } else {
+          aggBuilder += s"SUM(${distinct} ${quoteEachCols(column, dialect)})"
+        }
+      case Avg(column, dataType, isDistinct) =>
+        val distinct = if (isDistinct) "DISTINCT " else ""
+        dataTypeBuilder += dataType
+        if (!containsArithmeticOp(column)) {
+          aggBuilder += s"AVG(${distinct} ${quote(column)})"
+        } else {
+          aggBuilder += s"AVG(${distinct} ${quoteEachCols(column, dialect)})"
+        }
+      case Count(column, dataType, isDistinct) =>
+        val distinct = if (isDistinct) "DISTINCT " else ""
+        dataTypeBuilder += dataType
+        val col = if (column.equals("1")) column else quote(column)
+          aggBuilder += s"COUNT(${distinct} $col)"
+      case _ =>
+    }
+    (aggBuilder.result, dataTypeBuilder.result)
+  }
+
+  private def quoteEachCols (column: String, dialect: JdbcDialect): String = {

Review comment:
       nit: extra space

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/aggregates.scala
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.sources
+
+import org.apache.spark.sql.types.DataType
+
+case class Aggregation(aggregateExpressions: Seq[AggregateFunc],

Review comment:
       I think these need some docs since they are user-facing? and maybe some examples on how to handle `aggregateExpressions` and `groupByExpressions`. For the latter, should we also name it `groupByColumns`? 

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,132 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
-      val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
-        normalizedFilters.partition(SubqueryExpression.hasSubquery)
+          val aliasMap = getAliasMap(project)
+          var aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression =>
+                replaceAlias(agg, aliasMap).asInstanceOf[AggregateExpression]
+            }
+          }
+          aggregates = DataSourceStrategy.normalizeExprs(aggregates, relation.output)
+            .asInstanceOf[Seq[AggregateExpression]]
 
-      // `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)
-      val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery
+          val groupingExpressionsWithoutAlias = groupingExpressions.flatMap{ expr =>
+            expr.collect {
+              case e: Expression => replaceAlias(e, aliasMap)
+            }
+          }
+          val normalizedGroupingExpressions =
+            DataSourceStrategy.normalizeExprs(groupingExpressionsWithoutAlias, relation.output)
+
+          var newFilters = filters
+          aggregates.foreach(agg =>
+            if (agg.filter.nonEmpty)  {
+              // handle agg filter the same way as other filters
+              newFilters = newFilters :+ agg.filter.get
+            }
+          )
+
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, newFilters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)
+          } else { // only push down aggregate if all the filers can be push down
+            val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+              normalizedGroupingExpressions)
+
+            val (scan, output, normalizedProjects) =
+              processFilterAndColumn(scanBuilder, project, postScanFilters, relation)
+
+            logInfo(
+              s"""
+                 |Pushing operators to ${relation.name}
+                 |Pushed Filters: ${pushedFilters.mkString(", ")}
+                 |Post-Scan Filters: ${postScanFilters.mkString(",")}
+                 |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+                 |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+                 |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+            val wrappedScan = scan match {
+              case v1: V1Scan =>
+                val translated = newFilters.flatMap(DataSourceStrategy.translateFilter(_, true))
+                V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+              case _ => scan
+            }
+
+            if (aggregation.aggregateExpressions.isEmpty) {
+              Aggregate(groupingExpressions, resultExpressions, child)

Review comment:
       ditto

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,49 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {
+    case AttributeReference(name, _, _, _) => name
+    case Cast(child, _, _) => columnAsString (child)

Review comment:
       nit: extra space after `columnAsString`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,49 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {
+    case AttributeReference(name, _, _, _) => name
+    case Cast(child, _, _) => columnAsString (child)
+    case Add(left, right, _) =>
+      columnAsString(left) + " + " + columnAsString(right)
+    case Subtract(left, right, _) =>
+      columnAsString(left) + " - " + columnAsString(right)
+    case Multiply(left, right, _) =>
+      columnAsString(left) + " * " + columnAsString(right)
+    case Divide(left, right, _) =>
+      columnAsString(left) + " / " + columnAsString(right)
+    case CheckOverflow(child, _, _) => columnAsString (child)
+    case PromotePrecision(child) => columnAsString (child)
+    case _ => ""
+  }
+
+  protected[sql] def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+    aggregates.aggregateFunction match {
+      case min: aggregate.Min =>
+        val colName = columnAsString(min.child)
+        if (colName.nonEmpty) Some(Min(colName, min.dataType)) else None
+      case max: aggregate.Max =>
+        val colName = columnAsString(max.child)
+        if (colName.nonEmpty) Some(Max(colName, max.dataType)) else None
+      case avg: aggregate.Average =>
+        val colName = columnAsString(avg.child)
+        if (colName.nonEmpty) Some(Avg(colName, avg.dataType, aggregates.isDistinct)) else None
+      case sum: aggregate.Sum =>
+        val colName = columnAsString(sum.child)
+        if (colName.nonEmpty) Some(Sum(colName, sum.dataType, aggregates.isDistinct)) else None
+      case count: aggregate.Count =>
+        val columnName = count.children.head match {
+          case Literal(_, _) => "1"

Review comment:
       why this is "1"? also should we check if there is more than one elements in `children`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,41 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {

Review comment:
       +1. It also seems strange to convert binary expression into a "magic" string form that is (seems) special to JDBC datasources.  
   
   I also wonder if we should handle nested columns the same way as `PushableColumnBase`

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala
##########
@@ -70,6 +72,43 @@ object PushDownUtils extends PredicateHelper {
     }
   }
 
+    /**
+     * Pushes down aggregates to the data source reader
+     *
+     * @return pushed aggregation.
+     */
+    def pushAggregates(
+        scanBuilder: ScanBuilder,
+        aggregates: Seq[AggregateExpression],
+        groupBy: Seq[Expression]): Aggregation = {
+
+      def columnAsString(e: Expression): String = e match {
+        case AttributeReference(name, _, _, _) => name
+        case _ => ""
+      }
+
+      scanBuilder match {
+        case r: SupportsPushDownAggregates =>
+          val translatedAggregates = mutable.ArrayBuffer.empty[sources.AggregateFunc]
+
+          for (aggregateExpr <- aggregates) {
+            val translated = DataSourceStrategy.translateAggregate(aggregateExpr)
+            if (translated.isEmpty) {
+              return Aggregation.empty
+            } else {
+              translatedAggregates += translated.get
+            }
+          }
+          val groupByCols = groupBy.map(columnAsString(_))
+          if (!groupByCols.exists(_.isEmpty)) {
+            r.pushAggregation(Aggregation(translatedAggregates, groupByCols))
+          }

Review comment:
       what about "else" branch? perhaps we should revise this code to:
   ```scala
           case r: SupportsPushDownAggregates =>
             val translatedAggregates = aggregates.map(DataSourceStrategy.translateAggregate)
             val translatedGroupBys = groupBy.map(columnAsString)
   
             if (translatedAggregates.exists(_.isEmpty) || translatedGroupBys.exists(_.isEmpty)) {
               Aggregation.empty
             } else {
               r.pushAggregation(Aggregation(translatedAggregates.flatten, translatedGroupBys))
               r.pushedAggregation
             }
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,132 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
-      val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
-        normalizedFilters.partition(SubqueryExpression.hasSubquery)
+          val aliasMap = getAliasMap(project)
+          var aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression =>
+                replaceAlias(agg, aliasMap).asInstanceOf[AggregateExpression]
+            }
+          }
+          aggregates = DataSourceStrategy.normalizeExprs(aggregates, relation.output)
+            .asInstanceOf[Seq[AggregateExpression]]
 
-      // `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)
-      val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery
+          val groupingExpressionsWithoutAlias = groupingExpressions.flatMap{ expr =>
+            expr.collect {
+              case e: Expression => replaceAlias(e, aliasMap)
+            }
+          }
+          val normalizedGroupingExpressions =
+            DataSourceStrategy.normalizeExprs(groupingExpressionsWithoutAlias, relation.output)
+
+          var newFilters = filters
+          aggregates.foreach(agg =>
+            if (agg.filter.nonEmpty)  {
+              // handle agg filter the same way as other filters
+              newFilters = newFilters :+ agg.filter.get
+            }
+          )
+
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, newFilters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)
+          } else { // only push down aggregate if all the filers can be push down
+            val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+              normalizedGroupingExpressions)
+
+            val (scan, output, normalizedProjects) =
+              processFilterAndColumn(scanBuilder, project, postScanFilters, relation)
+
+            logInfo(
+              s"""
+                 |Pushing operators to ${relation.name}
+                 |Pushed Filters: ${pushedFilters.mkString(", ")}
+                 |Post-Scan Filters: ${postScanFilters.mkString(",")}
+                 |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+                 |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+                 |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+            val wrappedScan = scan match {
+              case v1: V1Scan =>
+                val translated = newFilters.flatMap(DataSourceStrategy.translateFilter(_, true))
+                V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+              case _ => scan
+            }
+
+            if (aggregation.aggregateExpressions.isEmpty) {
+              Aggregate(groupingExpressions, resultExpressions, child)
+            } else {
+              val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+              for (i <- 0 until aggregates.length) {
+                aggOutputBuilder += AttributeReference(
+                  aggregation.aggregateExpressions(i).toString, aggregates(i).dataType)()

Review comment:
       hmm is this correct?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,132 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
-      val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
-        normalizedFilters.partition(SubqueryExpression.hasSubquery)
+          val aliasMap = getAliasMap(project)
+          var aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression =>
+                replaceAlias(agg, aliasMap).asInstanceOf[AggregateExpression]
+            }
+          }
+          aggregates = DataSourceStrategy.normalizeExprs(aggregates, relation.output)
+            .asInstanceOf[Seq[AggregateExpression]]
 
-      // `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)
-      val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery
+          val groupingExpressionsWithoutAlias = groupingExpressions.flatMap{ expr =>
+            expr.collect {
+              case e: Expression => replaceAlias(e, aliasMap)
+            }
+          }
+          val normalizedGroupingExpressions =
+            DataSourceStrategy.normalizeExprs(groupingExpressionsWithoutAlias, relation.output)
+
+          var newFilters = filters
+          aggregates.foreach(agg =>
+            if (agg.filter.nonEmpty)  {
+              // handle agg filter the same way as other filters
+              newFilters = newFilters :+ agg.filter.get
+            }
+          )
+
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, newFilters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)
+          } else { // only push down aggregate if all the filers can be push down
+            val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+              normalizedGroupingExpressions)
+
+            val (scan, output, normalizedProjects) =
+              processFilterAndColumn(scanBuilder, project, postScanFilters, relation)
+
+            logInfo(
+              s"""
+                 |Pushing operators to ${relation.name}
+                 |Pushed Filters: ${pushedFilters.mkString(", ")}
+                 |Post-Scan Filters: ${postScanFilters.mkString(",")}
+                 |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+                 |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+                 |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+            val wrappedScan = scan match {
+              case v1: V1Scan =>
+                val translated = newFilters.flatMap(DataSourceStrategy.translateFilter(_, true))
+                V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+              case _ => scan
+            }
+
+            if (aggregation.aggregateExpressions.isEmpty) {
+              Aggregate(groupingExpressions, resultExpressions, child)
+            } else {
+              val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+              for (i <- 0 until aggregates.length) {
+                aggOutputBuilder += AttributeReference(
+                  aggregation.aggregateExpressions(i).toString, aggregates(i).dataType)()
+              }
+              groupingExpressions.foreach{
+                case a@AttributeReference(_, _, _, _) => aggOutputBuilder += a
+                case _ =>
+              }
+              val aggOutput = aggOutputBuilder.result
+
+              val r = buildLogicalPlan(aggOutput, relation, wrappedScan, aggOutput,
+                normalizedProjects, postScanFilters)
+              val plan = Aggregate(groupingExpressions, resultExpressions, r)
+
+              var i = 0
+              plan.transformExpressions {
+                case agg: AggregateExpression =>
+                  i += 1
+                  val aggFunction: aggregate.AggregateFunction = {
+                    if (agg.aggregateFunction.isInstanceOf[aggregate.Max]) {

Review comment:
       pattern matching?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
##########
@@ -273,6 +273,16 @@ trait PrunedFilteredScan {
   def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row]
 }
 
+/**
+ * @since 3.1.0
+ */
+trait PrunedFilteredAggregateScan {

Review comment:
       it's a bit strange that this is a DSv1 API but is only used by DSv2 JDBC scan? is it possible that a V1 data source implements this and goes through the V1 code path (i.e., through `DataSourceStrategy`)?




-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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] viirya commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownAggregates.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.connector.read;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.Aggregation;
+
+/**
+ * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
+ * push down aggregates to the data source.
+ *
+ * @since 3.1.0

Review comment:
       Maybe 3.2.0 now.

##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownAggregates.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.connector.read;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.Aggregation;
+
+/**
+ * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
+ * push down aggregates to the data source.
+ *
+ * @since 3.1.0
+ */
+@Evolving
+public interface SupportsPushDownAggregates extends ScanBuilder {
+
+  /**
+   * Pushes down Aggregation to datasource.
+   * The Aggregation can be pushed down only if all the Aggregate Functions can
+   * be pushed down.
+   */
+  void pushAggregation(Aggregation aggregation);
+
+  /**
+   * Returns the aggregates that are pushed to the data source via

Review comment:
       I think this returns `Aggregation`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,122 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, filters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)
+          } else { // only push down aggregate of all the filers can be push down
+            val aliasMap = getAliasMap(project)
+            var aggregates = resultExpressions.flatMap { expr =>
+              expr.collect {
+                case agg: AggregateExpression =>
+                  replaceAlias(agg, aliasMap).asInstanceOf[AggregateExpression]
+              }
+            }
+            aggregates = DataSourceStrategy.normalizeExprs(aggregates, relation.output)
+              .asInstanceOf[Seq[AggregateExpression]]
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
-      val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
-        normalizedFilters.partition(SubqueryExpression.hasSubquery)
+            val groupingExpressionsWithoutAlias = groupingExpressions.flatMap{ expr =>
+              expr.collect {
+                case a: AttributeReference => replaceAlias(a, aliasMap)
+              }
+            }
+            val normalizedgroupingExpressions =

Review comment:
       normalizedGroupingExpressions

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,41 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {

Review comment:
       For predicate pushdown, seems we simplify the cases to handle by only looking at column name.
   
   This covers a lot of cases but also makes it easy to break. We can begin with simplest case and add more supports later.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,122 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, filters, relation)

Review comment:
       I can see there is a dependency between filter pushdown and aggregate pushdown. As we need to check if all filters are pushed down.
   
   
   I think an alternative approach is to not touch filter pushdown, but to check if filter pushdown is happened and there is still `Filter` on top of the scan relation.
   
   I feel that can simplify the code here. And we don't need to call `pushDownFilter` twice for aggregate and filter.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -181,21 +243,49 @@ private[jdbc] class JDBCRDD(
     filters: Array[Filter],
     partitions: Array[Partition],
     url: String,
-    options: JDBCOptions)
+    options: JDBCOptions,
+    aggregation: Aggregation = Aggregation.empty)
   extends RDD[InternalRow](sc, Nil) {
 
   /**
    * Retrieve the list of partitions corresponding to this RDD.
    */
   override def getPartitions: Array[Partition] = partitions
 
+  private var updatedSchema: StructType = new StructType()
+
   /**
    * `columns`, but as a String suitable for injection into a SQL query.
    */
   private val columnList: String = {
+    val (compiledAgg, aggDataType) =
+      JDBCRDD.compileAggregates(aggregation.aggregateExpressions, JdbcDialects.get(url))
     val sb = new StringBuilder()
-    columns.foreach(x => sb.append(",").append(x))
-    if (sb.isEmpty) "1" else sb.substring(1)
+    if (compiledAgg.length == 0) {
+      updatedSchema = schema
+      columns.foreach(x => sb.append(",").append(x))
+    } else {
+      getAggregateColumnsList(sb, compiledAgg, aggDataType)
+    }
+    if (sb.length == 0) "1" else sb.substring(1)
+  }
+
+  private def getAggregateColumnsList(

Review comment:
       Shall we add a comment here to explain what `getAggregateColumnsList` does and why it is needed?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -700,6 +704,41 @@ object DataSourceStrategy
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  private def columnAsString(e: Expression): String = e match {

Review comment:
       Let's wait for others. See if there is any other voices.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -296,13 +398,15 @@ private[jdbc] class JDBCRDD(
 
     val myWhereClause = getWhereClause(part)
 
-    val sqlText = s"SELECT $columnList FROM ${options.tableOrQuery} $myWhereClause"
+    val sqlText = s"SELECT $columnList FROM ${options.tableOrQuery} $myWhereClause" +
+      s" $getGroupByClause"
     stmt = conn.prepareStatement(sqlText,
         ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)
     stmt.setFetchSize(options.fetchSize)
     stmt.setQueryTimeout(options.queryTimeout)
     rs = stmt.executeQuery()
-    val rowsIterator = JdbcUtils.resultSetToSparkInternalRows(rs, schema, inputMetrics)
+

Review comment:
       unnecessary change?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
##########
@@ -181,21 +243,49 @@ private[jdbc] class JDBCRDD(
     filters: Array[Filter],
     partitions: Array[Partition],
     url: String,
-    options: JDBCOptions)
+    options: JDBCOptions,
+    aggregation: Aggregation = Aggregation.empty)
   extends RDD[InternalRow](sc, Nil) {
 
   /**
    * Retrieve the list of partitions corresponding to this RDD.
    */
   override def getPartitions: Array[Partition] = partitions
 
+  private var updatedSchema: StructType = new StructType()
+
   /**
    * `columns`, but as a String suitable for injection into a SQL query.
    */
   private val columnList: String = {
+    val (compiledAgg, aggDataType) =
+      JDBCRDD.compileAggregates(aggregation.aggregateExpressions, JdbcDialects.get(url))
     val sb = new StringBuilder()
-    columns.foreach(x => sb.append(",").append(x))
-    if (sb.isEmpty) "1" else sb.substring(1)
+    if (compiledAgg.length == 0) {
+      updatedSchema = schema
+      columns.foreach(x => sb.append(",").append(x))
+    } else {
+      getAggregateColumnsList(sb, compiledAgg, aggDataType)

Review comment:
       `columns` is empty for this case?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,122 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, filters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)
+          } else { // only push down aggregate of all the filers can be push down

Review comment:
       of -> if

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,122 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
-object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+object V2ScanRelationPushDown extends Rule[LogicalPlan] with AliasHelper {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
-      val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val (pushedFilters, postScanFilters) = pushDownFilter(scanBuilder, filters, relation)
+          if (postScanFilters.nonEmpty) {
+            Aggregate(groupingExpressions, resultExpressions, child)
+          } else { // only push down aggregate of all the filers can be push down
+            val aliasMap = getAliasMap(project)
+            var aggregates = resultExpressions.flatMap { expr =>
+              expr.collect {
+                case agg: AggregateExpression =>
+                  replaceAlias(agg, aliasMap).asInstanceOf[AggregateExpression]
+              }
+            }
+            aggregates = DataSourceStrategy.normalizeExprs(aggregates, relation.output)
+              .asInstanceOf[Seq[AggregateExpression]]
 
-      val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
-      val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
-        normalizedFilters.partition(SubqueryExpression.hasSubquery)
+            val groupingExpressionsWithoutAlias = groupingExpressions.flatMap{ expr =>
+              expr.collect {
+                case a: AttributeReference => replaceAlias(a, aliasMap)
+              }
+            }
+            val normalizedgroupingExpressions =
+              DataSourceStrategy.normalizeExprs(groupingExpressionsWithoutAlias, relation.output)
 
-      // `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)
-      val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery
+            val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+              normalizedgroupingExpressions)
+
+            val (scan, output, normalizedProjects) =
+              processFilerAndColumn(scanBuilder, project, postScanFilters, relation)
+
+            logInfo(
+              s"""
+                 |Pushing operators to ${relation.name}
+                 |Pushed Filters: ${pushedFilters.mkString(", ")}
+                 |Post-Scan Filters: ${postScanFilters.mkString(",")}
+                 |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+                 |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+                 |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+            val wrappedScan = scan match {
+              case v1: V1Scan =>
+                val translated = filters.flatMap(DataSourceStrategy.translateFilter(_, true))
+                V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+              case _ => scan
+            }
+            if (aggregation.aggregateExpressions.isEmpty) {
+              val plan = buildLogicalPlan(project, relation, wrappedScan, output,
+                normalizedProjects, postScanFilters)
+              Aggregate(groupingExpressions, resultExpressions, plan)
+            } else {
+              val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+              for (i <- 0 until aggregates.length) {
+                aggOutputBuilder += AttributeReference(
+                  aggregation.aggregateExpressions(i).toString, aggregates(i).dataType)()
+              }
+              for (groupBy <- groupingExpressions) {
+                aggOutputBuilder += groupBy.asInstanceOf[AttributeReference]
+              }

Review comment:
       `groupingExpressions` is `Seq[Expression]`, are we sure they all `AttributeReference`?




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128545 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128545/testReport)** for PR 29695 at commit [`a30fd6d`](https://github.com/apache/spark/commit/a30fd6d38562773c6c525980c1a087c5fc915423).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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] rf972 commented on pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   First, thank you for this pull request!  We have found it very useful, and we are very excited to use this support to help enable aggregate pushdown in our own V2 datasource.
   
   We have looked into evaluating this code with TPCH, since we believe spark will see great gains with aggregate push down.
   One of the TPCH queries Q06 is:  select sum(l_extendedprice*l_discount) as revenue
   
   We saw issues with the aggregates and product in our testing.  To help illustrate this issue, we added a similar case with sum and product to JDBCV2Suite's test("scan with aggregate push-down")
   val df6 = sql("select MIN(SALARY) * MIN(BONUS) FROM h2.test.employee")
   df6.explain(true)
   
   Below is the error that we get.  
   [info]   java.lang.ArrayIndexOutOfBoundsException: Index 1 out of bounds for length 1
   [info]   at org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$$anonfun$apply$1$$anonfun$1.applyOrElse(V2ScanRelationPushDown.scala:116)
   [info]   at org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$$anonfun$apply$1$$anonfun$1.applyOrElse(V2ScanRelationPushDown.scala:109)
   [info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:318)
   ...
   
   val aggFunction: aggregate.AggregateFunction = {
   if (agg.aggregateFunction.isInstanceOf[aggregate.Max]) {
       aggregate.Max(aggOutput(i - 1))
   } else if (agg.aggregateFunction.isInstanceOf[aggregate.Min]) {
       aggregate.Min(aggOutput(i - 1)) <<<<<<<<<<<<<<<<<<<<<<<<<<<<<<< exception here
   
   Note that in this case aggOutput has just one item aggOutput[0] = (min(salary) *min(bonus))
   
   We are really interested in getting this case working and can help out with evaluation of a fix or even putting a fix together.  Any thoughts on a potential solution would be appreciated.  Thanks !
   


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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] maropu commented on a change in pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       For example, if `t` exists in a database side, it seems we can get different answers with/without pushdown?
   ```
   scala> Seq(0.6, 0.7, 0.8).toDF("v").write.saveAsTable("t")
   scala> sql("select sum(cast(v as int)) from t").show()
   +-------------------+
   |sum(CAST(v AS INT))|
   +-------------------+
   |                  0|
   +-------------------+
   ```




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128925 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128925/testReport)** for PR 29695 at commit [`adf1588`](https://github.com/apache/spark/commit/adf1588218a94e88fda7b7a90e824215b363cfcf).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131444 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131444/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131179 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131179/testReport)** for PR 29695 at commit [`da0be52`](https://github.com/apache/spark/commit/da0be52b1fbab6368bfdd8c61cfabb71b20625b0).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131404 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131404/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       I guess this is probably OK?
   For example, if I have `sum(cast(SALARY as bigInt))` here, I will remove cast and push down `sum(SALARY)` to data source. Then I will cast the output of `sum(SALARY)` to bigInt. 

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {
+        case AttributeReference(name, _, _, _) => name
+        case _ => ""
+      }
+      case _ => ""
+    }
+
+    aggregates.aggregateFunction match {

Review comment:
       I will need to change the following to add `isDistinct` and `filter`. Also change `translateAggregate` accordingly. When push down the aggregates, need to check the filter to make sure it can be pushed down too.
   ```
   case class Avg(column: String, isDistinct: Boolean, filter: Option[Filter]) extends AggregateFunc
   
   case class Min(column: String, isDistinct: Boolean, filter: Option[Filter]) extends AggregateFunc
   
   case class Max(column: String, isDistinct: Boolean, filter: Option[Filter]) extends AggregateFunc
   
   case class Sum(column: String, isDistinct: Boolean, filter: Option[Filter]) extends 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.

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 pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   also cc @cloud-fan @maropu @MaxGekk 
   I did some initial work for aggregate push down. There are still quite some places to be improved, but could you please take a look to see if this is OK? Thanks!


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
##########
@@ -264,6 +265,19 @@ private[sql] case class JDBCRelation(
     }
   }
 
+  override def unhandledAggregates(aggregates: Array[AggregateFunc]):
+    Array[AggregateFunc] = {
+    if (jdbcOptions.pushDownAggregate) {
+      if (JDBCRDD.compileAggregates(aggregates, JdbcDialects.get(jdbcOptions.url)).isEmpty) {
+        aggregates
+      } else {
+        Array.empty[AggregateFunc]
+      }
+    } else {
+      aggregates
+    }
+  }

Review comment:
       will remove this method




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #128673 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128673/testReport)** for PR 29695 at commit [`c45a2b6`](https://github.com/apache/spark/commit/c45a2b643f2e83af14f6b0584c5c76cf1e5af4c0).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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] maropu commented on a change in pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/aggregates.scala
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.sources
+
+case class Aggregation(aggregateExpressions: Seq[AggregateFunc],
+                       groupByExpressions: Seq[String])
+
+abstract class AggregateFunc
+
+// Todo: add Count
+
+case class Avg(column: String) extends AggregateFunc

Review comment:
       We can simply push down `Avg` aggregates? In Spark, partial aggregate output is a sequence of `(sum, count)`, but the output of scans having `Avg` is a sequence of partial averaged values. This can affect result precision, I think.




----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   retest this please


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/aggregates.scala
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.sources
+
+case class Aggregation(aggregateExpressions: Seq[AggregateFunc],
+                       groupByExpressions: Seq[String])
+
+abstract class AggregateFunc
+
+// Todo: add Count
+
+case class Avg(column: String) extends AggregateFunc

Review comment:
       I will follow Spark implementation and push down sum and count




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131444 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131444/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   **[Test build #136600 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136600/testReport)** for PR 29695 at commit [`782a0a8`](https://github.com/apache/spark/commit/782a0a827fbb1e60b8673d533cbda618df65dc96).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Count(children: Seq[Expression]) extends CountBase(children)`
     * `abstract class CountBase(children: Seq[Expression]) extends DeclarativeAggregate `
     * `case class PushDownCount(children: Seq[Expression]) extends CountBase(children) `


-- 
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 pull request #29695: [SPARK-22390][SPARK-32833][SQL] JDBC V2 Datasource aggregate push down

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


   @rf972 Thank you very much for helping us testing this! Really appreciate your help! 


-- 
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       If the col data type is float, I think we should remove the cast and pass it to database side. On database side, cast it to double:
   ```
   if (cast by Spark)
     remove cast from sum
     push down Sum(col) to database
     if (datatype of col is float)
       cast to Double, something like SELECT  Sum(CAST(col AS DOUBLE)) FROM table
   else // cast by user
     not remove cast and not push down aggregate
   ```




----------------------------------------------------------------
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:
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] rf972 commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -73,33 +77,25 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] {
               postScanFilters)
             Aggregate(groupingExpressions, resultExpressions, plan)
           } else {
-            val resultAttributes = resultExpressions.map(_.toAttribute)
-              .map ( e => e match { case a: AttributeReference => a })
-            var index = 0
             val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
-            for (a <- resultAttributes) {
-              val newName = if (a.name.contains("FILTER")) {
-                a.name.substring(0, a.name.indexOf("FILTER") - 1)
-              } else if (a.name.contains("DISTINCT")) {
-                a.name.replace("DISTINCT ", "")
-              } else {
-                a.name
-              }
-
-              aggOutputBuilder +=
-                a.copy(name = newName,
-                  dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
-                  qualifier = a.qualifier)
-              index += 1
+            for (a <- aggregates) {
+                aggOutputBuilder += AttributeReference(toPrettySQL(a), a.dataType)()
             }
             val aggOutput = aggOutputBuilder.result
 
-            var newOutput = aggOutput
-            for (col <- output) {
-              if (!aggOutput.exists(_.name.contains(col.name))) {
-                newOutput = col +: newOutput
+            val newOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (col1 <- output) {
+              var found = false
+              for (col2 <- aggOutput) {
+                  if (contains(col2.name, col1.name)) {

Review comment:
       Thanks @huaxingao for incorporating our changes in the patch !   
   Here is a brief overview of our TPCH testing with this patch.   We initially picked two queries from TPCH which should show immediate benefits from aggregate pushdown.  The Q01 test was our first choice, but we put that aside for now since it has UDFs and the count operator.  So now we are focusing on the Q06 test.  Our simulations show that the filter pushdown in Q06 reduces the data transfer (from storage/database) significantly (in our simulated case from 700 MB down to about 1.5 MB).  Our simulations also show that aggregate pushdown further reduces the data transfer from that 1.5 MB down to just 34 bytes.  
   The Q06 TPCH test requires support for an expression under an aggregate.  So something like sum(x * y).  We are actively working on this now, and once we finish adding this support (within a week or so, we hope), we will share the code and our actual TPCH results.  Also if you have any thoughts on the direction for adding this support that would be appreciated too.  Thanks!
   




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala
##########
@@ -69,6 +71,37 @@ object PushDownUtils extends PredicateHelper {
     }
   }
 
+    /**
+     * Pushes down aggregates to the data source reader
+     *
+     * @return pushed aggregates and post-scan aggregates.
+     */
+    def pushAggregates(scanBuilder: ScanBuilder, aggregates: Seq[AggregateExpression])
+      : (Seq[sources.AggregateFunction], Seq[AggregateExpression]) = {
+      scanBuilder match {
+        case r: SupportsPushDownAggregates =>
+          val translatedAggregates = mutable.ArrayBuffer.empty[sources.AggregateFunction]
+          // Catalyst aggregate expression that can't be translated to data source aggregates.
+          val untranslatableExprs = mutable.ArrayBuffer.empty[AggregateExpression]
+
+          for (aggregateExpr <- aggregates) {
+            val translated = DataSourceStrategy.translateAggregate(aggregateExpr)
+            if (translated.isEmpty) {
+              untranslatableExprs += aggregateExpr
+            } else {
+              translatedAggregates += translated.get
+            }
+          }
+
+          if (untranslatableExprs.isEmpty) r.pushAggregates(translatedAggregates.toArray)
+
+          // push down only if all the aggregates can be pushed down
+          if (!r.pushedAggregates.isEmpty) (r.pushedAggregates, Nil) else (Nil, aggregates)

Review comment:
       I actually do aggregate at Spark layer regardless if aggregate is pushed down or not




----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #136320 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136320/testReport)** for PR 29695 at commit [`ef4bab9`](https://github.com/apache/spark/commit/ef4bab92827642c9e9ed5b45767e96e98debba82).


-- 
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:
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] maropu commented on a change in pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -643,6 +647,34 @@ object DataSourceStrategy {
     (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
   }
 
+  def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
+
+    def columnAsString(e: Expression): String = e match {
+      case AttributeReference(name, _, _, _) => name
+      case Cast(child, _, _) => child match {

Review comment:
       For example, how about the fractional case? If `e` is float, we should not remove the cast but pass it into a database side for computing double-typed partial aggregated values?
   ```
   case Sum(e @ FractionalType()) if e.dataType != DoubleType => Sum(Cast(e, DoubleType))
   ```




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #132914 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132914/testReport)** for PR 29695 at commit [`18b9455`](https://github.com/apache/spark/commit/18b94552d70bd4a6d370e8e6b92a6f0f07b4649e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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] rf972 commented on a change in pull request #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -73,33 +77,25 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] {
               postScanFilters)
             Aggregate(groupingExpressions, resultExpressions, plan)
           } else {
-            val resultAttributes = resultExpressions.map(_.toAttribute)
-              .map ( e => e match { case a: AttributeReference => a })
-            var index = 0
             val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
-            for (a <- resultAttributes) {
-              val newName = if (a.name.contains("FILTER")) {
-                a.name.substring(0, a.name.indexOf("FILTER") - 1)
-              } else if (a.name.contains("DISTINCT")) {
-                a.name.replace("DISTINCT ", "")
-              } else {
-                a.name
-              }
-
-              aggOutputBuilder +=
-                a.copy(name = newName,
-                  dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
-                  qualifier = a.qualifier)
-              index += 1
+            for (a <- aggregates) {
+                aggOutputBuilder += AttributeReference(toPrettySQL(a), a.dataType)()
             }
             val aggOutput = aggOutputBuilder.result
 
-            var newOutput = aggOutput
-            for (col <- output) {
-              if (!aggOutput.exists(_.name.contains(col.name))) {
-                newOutput = col +: newOutput
+            val newOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (col1 <- output) {
+              var found = false
+              for (col2 <- aggOutput) {
+                  if (contains(col2.name, col1.name)) {

Review comment:
       Thanks @huaxingao for incorporating our suggestions into the patch !   
   Here is a brief overview of our TPCH benchmark testing strategy with this patch.   
   We initially picked two queries (Q01 and Q06) from TPCH which should show immediate benefits from aggregate pushdown.  
   The Q01 test was our first choice, but we put that aside, for now, since it has UDFs and the COUNT operator.  
   So now, we are focusing on the Q06 test.  Our simulation shows that the filter pushdown in Q06 significantly reduces the data transfer (from storage/database).
   In our simulated case from 720 MB down to about 1.5 MB.  Our simulation also shows that aggregate pushdown further reduces the data transfer from that 1.5 MB down to just 34 bytes.  
   The Q06 TPCH test requires support for an expression under an aggregate,  something like sum(x *y).  
   We are actively working on this now, and once we finish adding this support (within a week or so, we hope), we will share the code and our actual TPCH results.  
   If you have any thoughts or suggestions on the direction for adding this support we would appreciate it very much.  
   Thanks!
   
   




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -73,33 +77,25 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] {
               postScanFilters)
             Aggregate(groupingExpressions, resultExpressions, plan)
           } else {
-            val resultAttributes = resultExpressions.map(_.toAttribute)
-              .map ( e => e match { case a: AttributeReference => a })
-            var index = 0
             val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
-            for (a <- resultAttributes) {
-              val newName = if (a.name.contains("FILTER")) {
-                a.name.substring(0, a.name.indexOf("FILTER") - 1)
-              } else if (a.name.contains("DISTINCT")) {
-                a.name.replace("DISTINCT ", "")
-              } else {
-                a.name
-              }
-
-              aggOutputBuilder +=
-                a.copy(name = newName,
-                  dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
-                  qualifier = a.qualifier)
-              index += 1
+            for (a <- aggregates) {
+                aggOutputBuilder += AttributeReference(toPrettySQL(a), a.dataType)()
             }
             val aggOutput = aggOutputBuilder.result
 
-            var newOutput = aggOutput
-            for (col <- output) {
-              if (!aggOutput.exists(_.name.contains(col.name))) {
-                newOutput = col +: newOutput
+            val newOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (col1 <- output) {
+              var found = false
+              for (col2 <- aggOutput) {
+                  if (contains(col2.name, col1.name)) {

Review comment:
       @rf972 Thanks for sharing the TCPH tests results! I will fix the bug. I actually haven't done extensive tests yet.  I will do more tests after the Thanksgiving weekend. 




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131404 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131404/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130240/testReport)** for PR 29695 at commit [`5e92c25`](https://github.com/apache/spark/commit/5e92c25661df1245f3c75eac83a71eb75918665b).


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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] viirya commented on pull request #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   Thanks for pinging me. Let's sync with master and fix the compilation error?


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131179 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131179/testReport)** for PR 29695 at commit [`da0be52`](https://github.com/apache/spark/commit/da0be52b1fbab6368bfdd8c61cfabb71b20625b0).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -17,38 +17,117 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression}
+import scala.collection.mutable.ArrayBuilder
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 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.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.connector.read.{Scan, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.sources
+import org.apache.spark.sql.sources.{AggregateFunc, Aggregation}
 import org.apache.spark.sql.types.StructType
 
 object V2ScanRelationPushDown extends Rule[LogicalPlan] {
+
   import DataSourceV2Implicits._
 
   override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case Aggregate(groupingExpressions, resultExpressions, child) =>
+      child match {
+        case ScanOperation(project, filters, relation: DataSourceV2Relation) =>
+          val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options)
+          val aggregates = resultExpressions.flatMap { expr =>
+            expr.collect {
+              case agg: AggregateExpression => agg
+            }
+          }.distinct
+
+          val aggregation = PushDownUtils.pushAggregates(scanBuilder, aggregates,
+            groupingExpressions)
+
+          val (pushedFilters, postScanFilters, scan, output, normalizedProjects) =
+            processFilerAndColumn(scanBuilder, project, filters, relation)
+
+          logInfo(
+            s"""
+               |Pushing operators to ${relation.name}
+               |Pushed Filters: ${pushedFilters.mkString(", ")}
+               |Post-Scan Filters: ${postScanFilters.mkString(",")}
+               |Pushed Aggregate Functions: ${aggregation.aggregateExpressions.mkString(", ")}
+               |Pushed Groupby: ${aggregation.groupByExpressions.mkString(", ")}
+               |Output: ${output.mkString(", ")}
+             """.stripMargin)
+
+          val wrappedScan = scan match {
+            case v1: V1Scan =>
+              val translated = filters.flatMap(DataSourceStrategy.translateFilter(_, true))
+              V1ScanWrapper(v1, translated, pushedFilters, aggregation)
+            case _ => scan
+          }
+
+          if (aggregation.aggregateExpressions.isEmpty) {
+            val plan = buildLogicalPlan(project, relation, wrappedScan, output, normalizedProjects,
+              postScanFilters)
+            Aggregate(groupingExpressions, resultExpressions, plan)
+          } else {
+            val resultAttributes = resultExpressions.map(_.toAttribute)
+              .map ( e => e match { case a: AttributeReference => a })
+            var index = 0
+            val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (a <- resultAttributes) {
+              aggOutputBuilder +=
+                a.copy(dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
+                  qualifier = a.qualifier)
+              index += 1
+            }
+            val aggOutput = aggOutputBuilder.result
+
+            var newOutput = aggOutput
+            for (col <- output) {
+              if (!aggOutput.exists(_.name.contains(col.name))) {
+                newOutput = col +: newOutput
+              }
+            }
+
+            val r = buildLogicalPlan(newOutput, relation, wrappedScan, newOutput,
+              normalizedProjects, postScanFilters)
+            val plan = Aggregate(groupingExpressions, resultExpressions, r)
+
+            var i = 0
+            plan.transformExpressions {
+            case agg: AggregateExpression =>
+              val aggFunction: aggregate.AggregateFunction = {
+                i += 1
+                if (agg.aggregateFunction.isInstanceOf[aggregate.Max]) {
+                  aggregate.Max(aggOutput(i - 1))
+                } else if (agg.aggregateFunction.isInstanceOf[aggregate.Min]) {
+                  aggregate.Min(aggOutput(i - 1))
+                } else if (agg.aggregateFunction.isInstanceOf[aggregate.Average]) {
+                  aggregate.Average(aggOutput(i - 1))
+                } else if (agg.aggregateFunction.isInstanceOf[aggregate.Sum]) {
+                  aggregate.Sum(aggOutput(i - 1))
+                } else {
+                  agg.aggregateFunction
+                }
+              }
+              agg.transform {
+                case a: aggregate.AggregateFunction => aggFunction
+              }

Review comment:
       update optimized logical plan 
   before update
   ```
   Aggregate [max(id#18) AS max(id)#21, min(id#18) AS min(id)#22]
   +- RelationV2[ID#18] test.people
   ```
   after update
   ```
   Aggregate [max(max(ID)#77) AS max(ID)#72, min(min(ID)#78) AS min(ID)#73]
   +- RelationV2[max(ID)#77, min(ID)#78] test.people
   ```




----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #135329 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135329/testReport)** for PR 29695 at commit [`69813c7`](https://github.com/apache/spark/commit/69813c7de38ca7e123e7f7dbb3367560ea12ce52).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130239 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130239/testReport)** for PR 29695 at commit [`5e92c25`](https://github.com/apache/spark/commit/5e92c25661df1245f3c75eac83a71eb75918665b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Avg(column: String, isDistinct: Boolean, filter: Option[Filter]) extends AggregateFunc`
     * `case class Min(column: String, isDistinct: Boolean, filter: Option[Filter]) extends AggregateFunc`
     * `case class Max(column: String, isDistinct: Boolean, filter: Option[Filter]) extends AggregateFunc`
     * `case class Sum(column: String, isDistinct: Boolean, filter: Option[Filter]) extends 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.

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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #131430 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131430/testReport)** for PR 29695 at commit [`0be3c95`](https://github.com/apache/spark/commit/0be3c953dfb8e7f4b483500c30e01a58eefbb4e5).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


-- 
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


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


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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






----------------------------------------------------------------
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:
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 #29695: [SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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


   **[Test build #130238 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130238/testReport)** for PR 29695 at commit [`bafdba4`](https://github.com/apache/spark/commit/bafdba4d0a932417cac1ba4975549a8d9f5423fa).


----------------------------------------------------------------
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:
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 #29695: [SPARK-22390][SPARK-32833][SQL] [WIP]JDBC V2 Datasource aggregate push down

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
##########
@@ -73,33 +77,25 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] {
               postScanFilters)
             Aggregate(groupingExpressions, resultExpressions, plan)
           } else {
-            val resultAttributes = resultExpressions.map(_.toAttribute)
-              .map ( e => e match { case a: AttributeReference => a })
-            var index = 0
             val aggOutputBuilder = ArrayBuilder.make[AttributeReference]
-            for (a <- resultAttributes) {
-              val newName = if (a.name.contains("FILTER")) {
-                a.name.substring(0, a.name.indexOf("FILTER") - 1)
-              } else if (a.name.contains("DISTINCT")) {
-                a.name.replace("DISTINCT ", "")
-              } else {
-                a.name
-              }
-
-              aggOutputBuilder +=
-                a.copy(name = newName,
-                  dataType = aggregates(index).dataType)(exprId = NamedExpression.newExprId,
-                  qualifier = a.qualifier)
-              index += 1
+            for (a <- aggregates) {
+                aggOutputBuilder += AttributeReference(toPrettySQL(a), a.dataType)()
             }
             val aggOutput = aggOutputBuilder.result
 
-            var newOutput = aggOutput
-            for (col <- output) {
-              if (!aggOutput.exists(_.name.contains(col.name))) {
-                newOutput = col +: newOutput
+            val newOutputBuilder = ArrayBuilder.make[AttributeReference]
+            for (col1 <- output) {
+              var found = false
+              for (col2 <- aggOutput) {
+                  if (contains(col2.name, col1.name)) {

Review comment:
       Thanks @rf972. I committed the fix. 
   Are there any performance improvement with aggregate push down in your TPCH 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.

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