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 2019/12/31 11:08:55 UTC

[GitHub] [spark] beliefer opened a new pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

beliefer opened a new pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058
 
 
   ### What changes were proposed in this pull request?
   This PR is related to https://github.com/apache/spark/pull/26656.
   https://github.com/apache/spark/pull/26656 only support use FILTER clause on aggregate expression without DISTINCT.
   This PR will enhance this feature when one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause.
   Such as:
   ```
   select sum(distinct id) filter (where sex = 'man') from student;
   select class_id, sum(distinct id) filter (where sex = 'man') from student group by class_id;
   select count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student;
   select class_id, count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student group by class_id;
   select sum(distinct id), sum(distinct id) filter (where sex = 'man') from student;
   select class_id, sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
   select class_id, count(id), count(id) filter (where class_id = 1), sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
   ```
   This PR not support:
   ```
   select class_id, count(distinct sex), sum(distinct id) filter (where sex = 'man') from student group by class_id;
   select class_id, count(distinct sex) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student group by class_id;
   ```
   https://issues.apache.org/jira/browse/SPARK-30396 used for later.
   
   ### Why are the changes needed?
   No
   
   
   ### Does this PR introduce any user-facing change?
   No
   
   
   ### How was this patch tested?
   New UT
   

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-569942377
 
 
   **[Test build #115987 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115987/testReport)** for PR 27058 at commit [`0008bae`](https://github.com/apache/spark/commit/0008bae5ff27a6766b9ceb3bb067eac2e8e322dc).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-576748173
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364234459
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##########
 @@ -207,6 +207,29 @@ trait CheckAnalysis extends PredicateHelper {
                 s"of type ${condition.dataType.catalogString} is not a boolean.")
 
           case Aggregate(groupingExprs, aggregateExprs, child) =>
+            val aggExpressions = aggregateExprs.flatMap { e =>
+              e.collect {
+                case ae: AggregateExpression => ae
+              }
+            }
+            val aggWithDistinctAndFilters = aggExpressions.filter(e => e.isDistinct)
+            val aggGroups = aggWithDistinctAndFilters.groupBy { e =>
 
 Review comment:
   nit: `aggGroups` -> `distinctAggGroups`

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368805334
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,66 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat1_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat1_cnt2, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat1, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule serves two purposes:
+ * 1. Expand distinct aggregates which exists filter clause.
+ * 2. Rewrite when aggregate exists at least two distinct aggregates.
+ *
+ * The first child rule does the following things here:
+ * 1. Guaranteed to compute filter clause locally.
 
 Review comment:
   What does `compute filter clause locally` means?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570140108
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573077780
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116490/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570184658
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20825/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579133604
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22231/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577519811
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22041/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508203
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21680/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572864751
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578698892
 
 
   **[Test build #117436 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117436/testReport)** for PR 27058 at commit [`7a74aae`](https://github.com/apache/spark/commit/7a74aae09f8f696102c5b92b850d572d64fd9cb1).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575520166
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577670226
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367752087
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
 
 Review comment:
   How about `extractFiltersInDistinctAggregate`?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575129915
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116827/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578633620
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117434/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581008795
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22475/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577519805
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369315269
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,64 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) as cat2_cnt,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat2),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat2, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule consists of the two phases as follows.
+ *
+ * In the first phase, expands distinct aggregates which exists filter clause:
+ * 1. Guaranteed to compute filter clauses in the first aggregate locally.
+ * 2. The attributes referenced by different distinct aggregate expressions are likely to overlap,
+ *    and if no additional processing is performed, data loss will occur. To prevent this, we
+ *    generate new attributes and replace the original ones.
+ * 3. If we apply the first phase to distinct aggregate expressions which exists filter
+ *    clause, the aggregate after expand may have at least two distinct aggregates, so we need to
+ *    apply the second phase too. Please refer to the second phase for more details.
+ *
+ * In the second phase, rewrite when aggregate exists at least two distinct aggregates:
  * 1. Expand the data. There are three aggregation groups in this query:
 
 Review comment:
   It seems this statement refers to the second example? If so, the current position looks weird...

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578758706
 
 
   **[Test build #117442 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117442/testReport)** for PR 27058 at commit [`5c80418`](https://github.com/apache/spark/commit/5c804182b498c0e804cb0c3157d1dbd31d182e24).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r366171011
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -316,6 +362,86 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }.asInstanceOf[NamedExpression]
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
+    } else if (distinctAggGroups.size == 1) {
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            // Why do we need to construct the phantom id ?
+            // First, In order to reduce costs, it is better to handle the filter clause locally.
+            // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+            // If(id > 1) 'a else null first, and use the result as output.
+            // Second, If more than one DISTINCT aggregate expression uses the same column,
+            // We need to construct the phantom attributes so as the output not lost.
+            // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+            // attribute '_gen_distinct-1 and attribute '_gen_distinct-2 instead of two 'a.
+            // Note: We just need to illusion the expression with filter clause.
+            // The illusionary mechanism may result in multiple distinct aggregations uses
+            // different column, so we still need to call `rewrite`.
+            val phantomId = NamedExpression.newExprId.id
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"_gen_distinct_$phantomId", e.dataType, nullable = true)())
+            }
+            val exprAttrLookup = exprAttrs.toMap
+            val newChildren = af.children.map(c => exprAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = None)
+            // Expand projection
+            val projection = unfoldableChildren.map {
+              case e if filter.isDefined => If(filter.get, e, nullify(e))
+              case e => e
+            }
+            (projection, exprAttrs, (ae, aggExpr))
+        }
+        val rewriteDistinctAttrMap = rewriteDistinctOperatorMap.flatMap(_._2)
+        val distinctAggChildAttrs = rewriteDistinctAttrMap.map(_._2)
+        val allAggAttrs = regularAggChildAttrMap.map(_._2) ++ distinctAggChildAttrs
+        // Construct the aggregate input projection.
+        val rewriteDistinctProjections = rewriteDistinctOperatorMap.flatMap(_._1)
+        val rewriteAggProjections =
+          Seq((a.groupingExpressions ++ regularAggChildren ++ rewriteDistinctProjections))
+        val groupByMap = a.groupingExpressions.collect {
+          case ne: NamedExpression => ne -> ne.toAttribute
+          case e => e -> AttributeReference(e.sql, e.dataType, e.nullable)()
+        }
+        val groupByAttrs = groupByMap.map(_._2)
+        // Construct the expand operator.
+        val expand = Expand(rewriteAggProjections, groupByAttrs ++ allAggAttrs, a.child)
+        val rewriteAggExprLookup =
+          (rewriteDistinctOperatorMap.map(_._3) ++ regularOperatorMap).toMap
+        val patchedAggExpressions = a.aggregateExpressions.map { e =>
+          e.transformDown {
+            case ae: AggregateExpression => rewriteAggExprLookup.getOrElse(ae, ae)
+          }.asInstanceOf[NamedExpression]
+        }
+        val expandAggregate = Aggregate(groupByAttrs, patchedAggExpressions, expand)
+        if (distinctAggExpressions.size > 1) {
+          rewrite(expandAggregate)
 
 Review comment:
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577519805
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] maropu commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572820000
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573040868
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574943784
 
 
   **[Test build #116803 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116803/testReport)** for PR 27058 at commit [`c6392db`](https://github.com/apache/spark/commit/c6392db3fbedd7442d27d263b052e0fd940f6d32).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r366171505
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,6 +118,52 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat1_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat1_cnt2, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
 
 Review comment:
   Probably, this is related to [the comment](https://github.com/apache/spark/pull/27058#discussion_r366139379). If we avoid the recursive call, I think we can have a chance to merge them.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575146628
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578633612
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572445726
 
 
   **[Test build #116370 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116370/testReport)** for PR 27058 at commit [`bcca92d`](https://github.com/apache/spark/commit/bcca92d14d13531bd02ab9f9eb19afe001f8754e).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575520076
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116907/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572543838
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116370/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572542938
 
 
   **[Test build #116370 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116370/testReport)** for PR 27058 at commit [`bcca92d`](https://github.com/apache/spark/commit/bcca92d14d13531bd02ab9f9eb19afe001f8754e).
    * This patch **fails SparkR 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575031809
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575034151
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21599/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575031741
 
 
   **[Test build #116825 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116825/testReport)** for PR 27058 at commit [`a83efcf`](https://github.com/apache/spark/commit/a83efcf57021167bf9829f9f1ee2039ea9e86213).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) `

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573140754
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-576645591
 
 
   **[Test build #117179 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117179/testReport)** for PR 27058 at commit [`7a6b019`](https://github.com/apache/spark/commit/7a6b0197c1f32f6ad0d843b688b8abcc97344935).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577670237
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117291/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579116033
 
 
   **[Test build #117467 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117467/testReport)** for PR 27058 at commit [`8f9626b`](https://github.com/apache/spark/commit/8f9626be355d52f0ec6e0ed6750d438f9b0b0866).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573077769
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573043743
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579032752
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578645101
 
 
   **[Test build #117436 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117436/testReport)** for PR 27058 at commit [`7a74aae`](https://github.com/apache/spark/commit/7a74aae09f8f696102c5b92b850d572d64fd9cb1).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574620963
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21553/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-569912064
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578633674
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369536759
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,64 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) as cat2_cnt,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat2),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat2, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule consists of the two phases as follows.
+ *
+ * In the first phase, expands distinct aggregates which exists filter clause:
+ * 1. Guaranteed to compute filter clauses in the first aggregate locally.
+ * 2. The attributes referenced by different distinct aggregate expressions are likely to overlap,
+ *    and if no additional processing is performed, data loss will occur. To prevent this, we
+ *    generate new attributes and replace the original ones.
+ * 3. If we apply the first phase to distinct aggregate expressions which exists filter
+ *    clause, the aggregate after expand may have at least two distinct aggregates, so we need to
+ *    apply the second phase too. Please refer to the second phase for more details.
+ *
+ * In the second phase, rewrite when aggregate exists at least two distinct aggregates:
  * 1. Expand the data. There are three aggregation groups in this query:
 
 Review comment:
   Really? I think the statements below point to the second query: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala#L78-L119
   ```
    *    i. the non-distinct group;
    *    ii. the distinct 'cat1 group;
    *    iii. the distinct 'cat2 group.
   ```
   

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575028437
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575034151
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21599/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581022469
 
 
   Build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367727323
 
 

 ##########
 File path: sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out
 ##########
 @@ -444,10 +444,29 @@ NULL	1
 
 
 -- !query 34
-select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col)
+select ten, sum(distinct four) filter (where string(four) like '123') from onek a
+group by rollup(ten)
 -- !query 34 schema
-struct<count(1):bigint>
+struct<ten:int,sum(DISTINCT four) FILTER (WHERE CAST(four AS STRING) LIKE 123):bigint>
 -- !query 34 output
 
 Review comment:
   Checked, too: https://github.com/postgres/postgres/blob/0fca3d0a4ec297bff5a5cb01dfe345e0f63d7d63/src/test/regress/expected/groupingsets.out#L822

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364580011
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
 
 Review comment:
   The illusionary mechanism may result in multiple distinct aggregations uses different column, so we still need to call `rewrite`. (e.g., `count(distinct phantom1-a) and count(distinct a phantom2-a)`)

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364515324
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"phantom$phantomId-${e.sql}", e.dataType, nullable = true)())
+            }
+            val exprAttrLookup = exprAttrs.toMap
+            val newChildren = af.children.map(c => exprAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = None)
+            // Expand projection
+            val projection = unfoldableChildren.map {
+              case e if filter.isDefined => If(filter.get, e, nullify(e))
+              case e => e
+            }
+            (projection, exprAttrs, (ae, aggExpr))
+        }
+        val rewriteDistinctAttrMap = rewriteDistinctOperatorMap.map(_._2).flatten
+        val distinctAggChildAttrs = rewriteDistinctAttrMap.map(_._2)
+        val allAggAttrs = (regularAggChildAttrMap.map(_._2) ++ distinctAggChildAttrs)
+        // Construct the aggregate input projection.
+        val rewriteDistinctProjections = rewriteDistinctOperatorMap.map(_._1).flatten
 
 Review comment:
   ditto: `.flatMap(_._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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574620963
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21553/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581008791
 
 
   Build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367741733
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
 
 Review comment:
   Can you describe more here? Its like `Replaces the existing references with new ones to avoid collisions between distinct and regular aggregate 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578758706
 
 
   **[Test build #117442 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117442/testReport)** for PR 27058 at commit [`5c80418`](https://github.com/apache/spark/commit/5c804182b498c0e804cb0c3157d1dbd31d182e24).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578645101
 
 
   **[Test build #117436 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117436/testReport)** for PR 27058 at commit [`7a74aae`](https://github.com/apache/spark/commit/7a74aae09f8f696102c5b92b850d572d64fd9cb1).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574620954
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574963605
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369338549
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +205,102 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // We need at least two distinct aggregates or a single distinct aggregate with a filter for
+    // this rule because aggregation strategy can handle a single distinct group without a filter.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregates(a)
+      rewriteDistinctAggregates(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregates(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Constructs pairs between old and new expressions for regular aggregates. Because we
+      // will construct a new aggregate, the children of the distinct aggregates will be
+      // changed to the generate ones, so we need creates new references to avoid collisions
+      // between distinct and regular aggregate children.
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369340309
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+          val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+          val filterOpt = filter.map(_.transform {
+            case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+          })
+          val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+          (ae, aggExpr)
+      }
+
+      // Setup expand for the distinct aggregate expressions.
+      val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val rewriteDistinctOperatorMap = distinctAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          // Why do we need to construct the phantom id ?
+          // First, In order to reduce costs, it is better to handle the filter clause locally.
+          // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+          // If(id > 1) 'a else null first, and use the result as output.
+          // Second, If more than one DISTINCT aggregate expression uses the same column,
+          // We need to construct the phantom attributes so as the output not lost.
+          // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+          // attribute '_gen_distinct-1 and attribute '_gen_distinct-2 instead of two 'a.
+          // Note: We just need to illusion the expression with filter clause.
+          // The illusionary mechanism may result in multiple distinct aggregations uses
+          // different column, so we still need to call `rewrite`.
 
 Review comment:
   OK. I will update it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577519423
 
 
   **[Test build #117281 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117281/testReport)** for PR 27058 at commit [`4fa0dd8`](https://github.com/apache/spark/commit/4fa0dd89ed36eefe8c7f12e39b23eb666a351673).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570250683
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578620862
 
 
   Build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572920132
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570117792
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572386876
 
 
   **[Test build #116345 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116345/testReport)** for PR 27058 at commit [`bcca92d`](https://github.com/apache/spark/commit/bcca92d14d13531bd02ab9f9eb19afe001f8754e).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-576646057
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508198
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367724105
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+          val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+          val filterOpt = filter.map(_.transform {
+            case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+          })
+          val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+          (ae, aggExpr)
+      }
+
+      // Setup expand for the distinct aggregate expressions.
+      val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val rewriteDistinctOperatorMap = distinctAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          // Why do we need to construct the phantom id ?
+          // First, In order to reduce costs, it is better to handle the filter clause locally.
+          // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+          // If(id > 1) 'a else null first, and use the result as output.
+          // Second, If more than one DISTINCT aggregate expression uses the same column,
+          // We need to construct the phantom attributes so as the output not lost.
+          // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+          // attribute '_gen_distinct-1 and attribute '_gen_distinct-2 instead of two 'a.
+          // Note: We just need to illusion the expression with filter clause.
+          // The illusionary mechanism may result in multiple distinct aggregations uses
+          // different column, so we still need to call `rewrite`.
+          val phantomId = NamedExpression.newExprId.id
+          val unfoldableChildren = af.children.filter(!_.foldable)
+          val exprAttrs = unfoldableChildren.map { e =>
+            (e, AttributeReference(s"_gen_distinct_$phantomId", e.dataType, nullable = true)())
 
 Review comment:
   nit: `phantomId` -> `exprId`

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572864514
 
 
   **[Test build #116445 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116445/testReport)** for PR 27058 at commit [`dd09645`](https://github.com/apache/spark/commit/dd096453c640335fe7af9f8b89eac296ee54615c).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573043256
 
 
   **[Test build #116491 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116491/testReport)** for PR 27058 at commit [`45d6971`](https://github.com/apache/spark/commit/45d69714a7a1bb6dc8a05a11a2b885174f5d3fca).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578633674
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579220034
 
 
   **[Test build #117472 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117472/testReport)** for PR 27058 at commit [`8f9626b`](https://github.com/apache/spark/commit/8f9626be355d52f0ec6e0ed6750d438f9b0b0866).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] amanomer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
amanomer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r362714875
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##########
 @@ -207,6 +207,29 @@ trait CheckAnalysis extends PredicateHelper {
                 s"of type ${condition.dataType.catalogString} is not a boolean.")
 
           case Aggregate(groupingExprs, aggregateExprs, child) =>
+            val aggExpressions = aggregateExprs.flatMap { e =>
+              e.collect {
+                case ae: AggregateExpression => ae
+              }
+            }
+            val aggWithDistinctAndFilters = aggExpressions.filter(e => e.isDistinct)
 
 Review comment:
   Why is this `aggWithDistinctAndFilters` since this is only checking for distinct?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575033417
 
 
   **[Test build #116827 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116827/testReport)** for PR 27058 at commit [`a83efcf`](https://github.com/apache/spark/commit/a83efcf57021167bf9829f9f1ee2039ea9e86213).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] maropu commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572906391
 
 
   Ah, also, can you put a simple explain example (about how to convert a plan with distinct aggregates) in the PR description? better to put how-to-fix in this pr there.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508244
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581022473
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117714/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578699422
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577519423
 
 
   **[Test build #117281 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117281/testReport)** for PR 27058 at commit [`4fa0dd8`](https://github.com/apache/spark/commit/4fa0dd89ed36eefe8c7f12e39b23eb666a351673).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578645866
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572387262
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570117654
 
 
   **[Test build #116012 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116012/testReport)** for PR 27058 at commit [`885e2f6`](https://github.com/apache/spark/commit/885e2f620a0d889f142ad3b1bef3d48dbcf28369).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572915374
 
 
   > btw, we need a different approach for supporting multiple distinct groups (SPARK-30396)? Why did you split the distinct support into two?
   
   This PR will support
   `select a, sum(distinct b) filter (where ...) from t group by a;`
   We only have one DISTINCT aggregate expr, so the columns where the aggregate function acting on is same.
   `select a, sum(distinct b) filter (where ...), count(distinct b) filter (where ...) from t group by a;`
   We have two DISTINCT aggregate exprs, but the columns where each aggregate function acting on is same.
   SPARK-30396 will support
   `select a, sum(distinct b) filter (where ...), count(distinct c) filter (where ...) from t group by a;`
   We have two DISTINCT aggregate exprs, the columns where each aggregate function acting on is different.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364545222
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"phantom$phantomId-${e.sql}", e.dataType, nullable = true)())
+            }
+            val exprAttrLookup = exprAttrs.toMap
+            val newChildren = af.children.map(c => exprAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = None)
+            // Expand projection
+            val projection = unfoldableChildren.map {
+              case e if filter.isDefined => If(filter.get, e, nullify(e))
+              case e => e
+            }
+            (projection, exprAttrs, (ae, aggExpr))
+        }
+        val rewriteDistinctAttrMap = rewriteDistinctOperatorMap.map(_._2).flatten
+        val distinctAggChildAttrs = rewriteDistinctAttrMap.map(_._2)
+        val allAggAttrs = (regularAggChildAttrMap.map(_._2) ++ distinctAggChildAttrs)
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-576981621
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21972/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575520076
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116907/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572874663
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116434/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575587787
 
 
   **[Test build #116939 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116939/testReport)** for PR 27058 at commit [`5462c0c`](https://github.com/apache/spark/commit/5462c0ce605cf95e608c945e4ca8bff358740f50).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578876431
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117442/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368940140
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
 
 Review comment:
   Currently, this pr only has end-to-end tests now. So, I think we need fine-grained tests, too, for the new logic in this rule.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368940374
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions. Because we will construct a new
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r365198561
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -316,6 +362,86 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }.asInstanceOf[NamedExpression]
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
+    } else if (distinctAggGroups.size == 1) {
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            // Why do we need to construct the phantom id ?
+            // First, In order to reduce costs, it is better to handle the filter clause locally.
+            // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+            // If(id > 1) 'a else null first, and use the result as output.
+            // Second, If more than one DISTINCT aggregate expression uses the same column,
+            // We need to construct the phantom attributes so as the output not lost.
+            // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+            // attribute 'phantom1-a and attribute 'phantom2-a instead of two 'a.
+            // Note: We just need to illusion the expression with filter clause.
+            // The illusionary mechanism may result in multiple distinct aggregations uses
+            // different column, so we still need to call `rewrite`.
+            val phantomId = i + 1
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"phantom$phantomId-${e.sql}", e.dataType, nullable = true)())
 
 Review comment:
   Just a example. It's ok to drop `_group`.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573043757
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21279/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367752087
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
 
 Review comment:
   How about `expandFiltersInDistinctAggregate`?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367741302
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
 
 Review comment:
   `val regularAggChildren = regularAggExpressions.flatMap(_.references)`?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572444344
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r371613436
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
 ##########
 @@ -165,20 +165,35 @@ package object dsl {
     }
 
     def rand(e: Long): Expression = Rand(e)
-    def sum(e: Expression): Expression = Sum(e).toAggregateExpression()
-    def sumDistinct(e: Expression): Expression = Sum(e).toAggregateExpression(isDistinct = true)
-    def count(e: Expression): Expression = Count(e).toAggregateExpression()
+    def sum(e: Expression, filter: Option[Expression] = None): Expression =
+      Sum(e).toAggregateExpression(false, filter)
+    def sumDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Sum(e).toAggregateExpression(true, filter)
+    def count(e: Expression, filter: Option[Expression] = None): Expression =
+      Count(e).toAggregateExpression(false, filter)
     def countDistinct(e: Expression*): Expression =
       Count(e).toAggregateExpression(isDistinct = true)
-    def approxCountDistinct(e: Expression, rsd: Double = 0.05): Expression =
-      HyperLogLogPlusPlus(e, rsd).toAggregateExpression()
-    def avg(e: Expression): Expression = Average(e).toAggregateExpression()
-    def first(e: Expression): Expression = new First(e).toAggregateExpression()
-    def last(e: Expression): Expression = new Last(e).toAggregateExpression()
-    def min(e: Expression): Expression = Min(e).toAggregateExpression()
-    def minDistinct(e: Expression): Expression = Min(e).toAggregateExpression(isDistinct = true)
-    def max(e: Expression): Expression = Max(e).toAggregateExpression()
-    def maxDistinct(e: Expression): Expression = Max(e).toAggregateExpression(isDistinct = true)
+    def countDistinct(filter: Option[Expression], e: Expression*): Expression =
+      Count(e).toAggregateExpression(true, filter)
+    def approxCountDistinct(
+        e: Expression,
+        rsd: Double = 0.05,
+        filter: Option[Expression] = None): Expression =
+      HyperLogLogPlusPlus(e, rsd).toAggregateExpression(true, filter)
+    def avg(e: Expression, filter: Option[Expression] = None): Expression =
+      Average(e).toAggregateExpression(false, filter)
+    def first(e: Expression, filter: Option[Expression] = None): Expression =
+      new First(e).toAggregateExpression(false, filter)
+    def last(e: Expression, filter: Option[Expression] = None): Expression =
+      new Last(e).toAggregateExpression(false, filter)
+    def min(e: Expression, filter: Option[Expression] = None): Expression =
+      Min(e).toAggregateExpression(false, filter)
+    def minDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Min(e).toAggregateExpression(true, filter)
+    def max(e: Expression, filter: Option[Expression] = None): Expression =
+      Max(e).toAggregateExpression(false, filter)
+    def maxDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Max(e).toAggregateExpression(true, filter)
     def upper(e: Expression): Expression = Upper(e)
 
 Review comment:
   Ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575031814
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116825/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572440680
 
 
   **[Test build #116345 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116345/testReport)** for PR 27058 at commit [`bcca92d`](https://github.com/apache/spark/commit/bcca92d14d13531bd02ab9f9eb19afe001f8754e).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367768976
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
 
 Review comment:
   `regularOperatorMap` -> `regularAggMap `?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-575028449
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21597/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574943784
 
 
   **[Test build #116803 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116803/testReport)** for PR 27058 at commit [`c6392db`](https://github.com/apache/spark/commit/c6392db3fbedd7442d27d263b052e0fd940f6d32).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581009195
 
 
   **[Test build #117714 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117714/testReport)** for PR 27058 at commit [`a9f8812`](https://github.com/apache/spark/commit/a9f88120f60273237a8bfd170a27132b71e4d659).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572446217
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367785540
 
 

 ##########
 File path: sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql
 ##########
 @@ -58,13 +62,18 @@ select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary),
 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id;
 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id;
 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id;
--- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT
--- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id;
--- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id;
--- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id;
--- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id;
--- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id;
--- select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id;
+select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id;
+select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id;
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-576982894
 
 
   **[Test build #117212 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117212/testReport)** for PR 27058 at commit [`c9e95c3`](https://github.com/apache/spark/commit/c9e95c33bc93e18316981704a776c07f8ffb12fb).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579088581
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367791466
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+          val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+          val filterOpt = filter.map(_.transform {
+            case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+          })
+          val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+          (ae, aggExpr)
+      }
+
+      // Setup expand for the distinct aggregate expressions.
+      val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val rewriteDistinctOperatorMap = distinctAggExprs.map {
 
 Review comment:
   `val (var1, var2, var3) = distinctAggExprs.map { ... }.unzip3`?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575516885
 
 
   **[Test build #116915 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116915/testReport)** for PR 27058 at commit [`5462c0c`](https://github.com/apache/spark/commit/5462c0ce605cf95e608c945e4ca8bff358740f50).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367753319
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
 
 Review comment:
   I want to reuse `rewriteDistinctAggregate` and the inner `Expand`.
   Merge them will lead to the code of `rewriteDistinctAggregate` more complex.
   I think it's not worth.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575501041
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581022473
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117714/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570188414
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116017/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575519997
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116915/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r365040943
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
 
 Review comment:
   In the current approach, it seems filter conditions are evaluated in not aggregate but expand. You meant, even in this case (the same aggregate group), we cannot use the aggregate filter logic in `AggregationIterator` implemented in your previous commit?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572946945
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575467747
 
 
   **[Test build #116895 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116895/testReport)** for PR 27058 at commit [`0ed5be0`](https://github.com/apache/spark/commit/0ed5be0b4e7edab5fef67c08488af7e85b999c40).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508198
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367756787
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+          val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+          val filterOpt = filter.map(_.transform {
+            case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+          })
+          val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+          (ae, aggExpr)
+      }
+
+      // Setup expand for the distinct aggregate expressions.
+      val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val rewriteDistinctOperatorMap = distinctAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          // Why do we need to construct the phantom id ?
+          // First, In order to reduce costs, it is better to handle the filter clause locally.
+          // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+          // If(id > 1) 'a else null first, and use the result as output.
+          // Second, If more than one DISTINCT aggregate expression uses the same column,
+          // We need to construct the phantom attributes so as the output not lost.
+          // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+          // attribute '_gen_distinct-1 and attribute '_gen_distinct-2 instead of two 'a.
+          // Note: We just need to illusion the expression with filter clause.
+          // The illusionary mechanism may result in multiple distinct aggregations uses
+          // different column, so we still need to call `rewrite`.
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367318890
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##########
 @@ -207,6 +207,26 @@ trait CheckAnalysis extends PredicateHelper {
                 s"of type ${condition.dataType.catalogString} is not a boolean.")
 
           case Aggregate(groupingExprs, aggregateExprs, child) =>
+            val distinctAggregateExprs = aggregateExprs.flatMap(_.collect {
+              case ae: AggregateExpression if ae.isDistinct => ae
+            })
+            val distinctAggGroups = distinctAggregateExprs.groupBy { e =>
+              val unfoldableChildren = e.aggregateFunction.children.filter(!_.foldable).toSet
+              if (unfoldableChildren.nonEmpty) {
+                unfoldableChildren
+              } else {
+                e.aggregateFunction.children.take(1).toSet
 
 Review comment:
   This PR is changed to implement multiple DISTINCT aggregate with Filter clause. I modified the title and description of this PR.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367786715
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574620588
 
 
   **[Test build #116781 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116781/testReport)** for PR 27058 at commit [`aa37870`](https://github.com/apache/spark/commit/aa37870ec36a8bf182c360a37c46d138d267f8a4).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579126684
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578618570
 
 
   **[Test build #117432 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117432/testReport)** for PR 27058 at commit [`71ba1f4`](https://github.com/apache/spark/commit/71ba1f46229cb9443658818b1f94b2973fbc37ce).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581009195
 
 
   **[Test build #117714 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117714/testReport)** for PR 27058 at commit [`a9f8812`](https://github.com/apache/spark/commit/a9f88120f60273237a8bfd170a27132b71e4d659).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578625016
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22193/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578633679
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117432/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364545988
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
 ##########
 @@ -192,8 +192,10 @@ class AnalysisErrorSuite extends AnalysisTest {
 
   errorTest(
     "DISTINCT and FILTER cannot be used in aggregate functions at the same time",
-    CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"),
-    "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil)
+    CatalystSqlParser.parsePlan("SELECT sum(DISTINCT b), " +
+      "count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"),
+    "When there are multiple DISTINCT aggregate expressions acting on different " +
+    "fields, any DISTINCT aggregate expression not allow use FILTER clause." :: Nil)
 
 Review comment:
   OK. I will add tests in group-by-filter.sql

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368940326
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions. Because we will construct a new
+      // aggregate, the children of the distinct aggregates will be changed to the generate
+      // ones, so we need creates new references to avoid collisions between distinct and
+      // regular aggregate children.
+      val regularAggExprs = regularAggExpressions.filter(_.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularAggMap = regularAggExprs.map {
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] maropu commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572936002
 
 
   Yea, I know that. What I just want to know is that we need a lot of code changes to support the thrid case (SPARK-30396 )? ISTM we can easily extend this approache (expland w/ filters in the first aggregation phase) for supporting that. If so, I think its ok to merge the two work in this single PR. Any techinical barrir to do so?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570185163
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364544828
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
 
 Review comment:
               // Why do we need to construct the phantom id ?
               // First, In order to reduce costs, it is better to handle the filter clause locally.
               // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
               // If(id > 1) 'a else null first, and use the result as output.
               // Second, If more than one DISTINCT aggregate expression uses the same column,
               // We need to construct the phantom attributes so as the output not lost.
               // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
               // attribute 'a and attribute 'phantom1-a instead of two 'a.
               // Note: We just need to illusion the expression with filter clause.
               // The illusionary mechanism may result in multiple distinct aggregations uses
               // different column, so we still need to call `rewrite`.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364536129
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##########
 @@ -207,6 +207,29 @@ trait CheckAnalysis extends PredicateHelper {
                 s"of type ${condition.dataType.catalogString} is not a boolean.")
 
           case Aggregate(groupingExprs, aggregateExprs, child) =>
+            val aggExpressions = aggregateExprs.flatMap { e =>
+              e.collect {
+                case ae: AggregateExpression => ae
+              }
+            }
+            val aggWithDistinctAndFilters = aggExpressions.filter(e => e.isDistinct)
+            val aggGroups = aggWithDistinctAndFilters.groupBy { e =>
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364236076
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##########
 @@ -207,6 +207,29 @@ trait CheckAnalysis extends PredicateHelper {
                 s"of type ${condition.dataType.catalogString} is not a boolean.")
 
           case Aggregate(groupingExprs, aggregateExprs, child) =>
+            val aggExpressions = aggregateExprs.flatMap { e =>
+              e.collect {
+                case ae: AggregateExpression => ae
+              }
+            }
+            val aggWithDistinctAndFilters = aggExpressions.filter(e => e.isDistinct)
+            val aggGroups = aggWithDistinctAndFilters.groupBy { e =>
+              val unfoldableChildren = e.aggregateFunction.children.filter(!_.foldable).toSet
+              if (unfoldableChildren.nonEmpty) {
+                unfoldableChildren
+              } else {
+                e.aggregateFunction.children.take(1).toSet
+              }
+            }
+            val useDistinctAndFilter = aggWithDistinctAndFilters.exists(_.filter.isDefined)
+            // TODO: SPARK-30396 When there are multiple DISTINCT aggregate expressions
+            // acting on different fields, any DISTINCT aggregate expression allows the use of
+            // the FILTER clause
+            if (aggGroups.size > 1 && useDistinctAndFilter) {
 
 Review comment:
   nit: how about inlining `useDistinctAndFilter `;
   ```
               if (aggGroups.size > 1 && aggWithDistinctAndFilters.exists(_.filter.isDefined)) {
   ```

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570140113
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116012/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364544828
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
 
 Review comment:
               // Why do we need to construct the phantom id ?
               // First, In order to reduce costs, it is better to handle the filter clause locally.
               // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
               // If(id > 1) 'a else null first, and use the result as output.
               // Second, If more than one DISTINCT aggregate expression uses the same column,
               // We need to construct the phantom attributes so as the output not lost.
               // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
               // attribute 'a and attribute 'phantom1-a instead of two 'a.
               // Note: We just need to illusion the expression with filter clause.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573043256
 
 
   **[Test build #116491 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116491/testReport)** for PR 27058 at commit [`45d6971`](https://github.com/apache/spark/commit/45d69714a7a1bb6dc8a05a11a2b885174f5d3fca).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574963615
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116803/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508244
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367766975
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
 
 Review comment:
   Since `rewrite` has less code now, can you inline this in `apply`?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367318890
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##########
 @@ -207,6 +207,26 @@ trait CheckAnalysis extends PredicateHelper {
                 s"of type ${condition.dataType.catalogString} is not a boolean.")
 
           case Aggregate(groupingExprs, aggregateExprs, child) =>
+            val distinctAggregateExprs = aggregateExprs.flatMap(_.collect {
+              case ae: AggregateExpression if ae.isDistinct => ae
+            })
+            val distinctAggGroups = distinctAggregateExprs.groupBy { e =>
+              val unfoldableChildren = e.aggregateFunction.children.filter(!_.foldable).toSet
+              if (unfoldableChildren.nonEmpty) {
+                unfoldableChildren
+              } else {
+                e.aggregateFunction.children.take(1).toSet
 
 Review comment:
   This PR is changed to implement multiple DISTINCT aggregate with Filter clause. I modified the title and description of this PR.
   We do not need these codes now.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367757765
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -331,6 +457,17 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     }
   }
 
+  /**
+   * Collect all aggregate expressions.
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575520172
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116895/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577669519
 
 
   **[Test build #117291 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117291/testReport)** for PR 27058 at commit [`4fa0dd8`](https://github.com/apache/spark/commit/4fa0dd89ed36eefe8c7f12e39b23eb666a351673).
    * This patch **fails SparkR 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572946419
 
 
   **[Test build #116480 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116480/testReport)** for PR 27058 at commit [`c65092b`](https://github.com/apache/spark/commit/c65092b9241d53a4b49909b2ac36c7c180db6aca).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578633293
 
 
   **[Test build #117432 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117432/testReport)** for PR 27058 at commit [`71ba1f4`](https://github.com/apache/spark/commit/71ba1f46229cb9443658818b1f94b2973fbc37ce).
    * This patch **fails due to an unknown error code, -9**.
    * This patch **does not merge 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579087842
 
 
   **[Test build #117456 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117456/testReport)** for PR 27058 at commit [`5c80418`](https://github.com/apache/spark/commit/5c804182b498c0e804cb0c3157d1dbd31d182e24).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579038043
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570139949
 
 
   **[Test build #116017 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116017/testReport)** for PR 27058 at commit [`a4fd143`](https://github.com/apache/spark/commit/a4fd143fa92cb569764b669e5c53f6dfd1f1ae6a).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577568841
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r366139379
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -316,6 +362,86 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }.asInstanceOf[NamedExpression]
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
+    } else if (distinctAggGroups.size == 1) {
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            // Why do we need to construct the phantom id ?
+            // First, In order to reduce costs, it is better to handle the filter clause locally.
+            // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+            // If(id > 1) 'a else null first, and use the result as output.
+            // Second, If more than one DISTINCT aggregate expression uses the same column,
+            // We need to construct the phantom attributes so as the output not lost.
+            // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+            // attribute '_gen_distinct-1 and attribute '_gen_distinct-2 instead of two 'a.
+            // Note: We just need to illusion the expression with filter clause.
+            // The illusionary mechanism may result in multiple distinct aggregations uses
+            // different column, so we still need to call `rewrite`.
+            val phantomId = NamedExpression.newExprId.id
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"_gen_distinct_$phantomId", e.dataType, nullable = true)())
+            }
+            val exprAttrLookup = exprAttrs.toMap
+            val newChildren = af.children.map(c => exprAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = None)
+            // Expand projection
+            val projection = unfoldableChildren.map {
+              case e if filter.isDefined => If(filter.get, e, nullify(e))
+              case e => e
+            }
+            (projection, exprAttrs, (ae, aggExpr))
+        }
+        val rewriteDistinctAttrMap = rewriteDistinctOperatorMap.flatMap(_._2)
+        val distinctAggChildAttrs = rewriteDistinctAttrMap.map(_._2)
+        val allAggAttrs = regularAggChildAttrMap.map(_._2) ++ distinctAggChildAttrs
+        // Construct the aggregate input projection.
+        val rewriteDistinctProjections = rewriteDistinctOperatorMap.flatMap(_._1)
+        val rewriteAggProjections =
+          Seq((a.groupingExpressions ++ regularAggChildren ++ rewriteDistinctProjections))
+        val groupByMap = a.groupingExpressions.collect {
+          case ne: NamedExpression => ne -> ne.toAttribute
+          case e => e -> AttributeReference(e.sql, e.dataType, e.nullable)()
+        }
+        val groupByAttrs = groupByMap.map(_._2)
+        // Construct the expand operator.
+        val expand = Expand(rewriteAggProjections, groupByAttrs ++ allAggAttrs, a.child)
+        val rewriteAggExprLookup =
+          (rewriteDistinctOperatorMap.map(_._3) ++ regularOperatorMap).toMap
+        val patchedAggExpressions = a.aggregateExpressions.map { e =>
+          e.transformDown {
+            case ae: AggregateExpression => rewriteAggExprLookup.getOrElse(ae, ae)
+          }.asInstanceOf[NamedExpression]
+        }
+        val expandAggregate = Aggregate(groupByAttrs, patchedAggExpressions, expand)
+        if (distinctAggExpressions.size > 1) {
+          rewrite(expandAggregate)
 
 Review comment:
   Can you avoid this recursive call? We cannot move this pre-processing logic (pushing down aggregate filters into projections in `Expand`) into the line 186? https://github.com/apache/spark/blob/1846b0261b84ce1bca079bc59fb4518bff910c18/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala#L186

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574620954
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573040878
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21277/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572543828
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367736938
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
 
 Review comment:
   nit: `regularFunChildren` -> `regularFunArguments`

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572446223
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21160/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574963520
 
 
   **[Test build #116803 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116803/testReport)** for PR 27058 at commit [`c6392db`](https://github.com/apache/spark/commit/c6392db3fbedd7442d27d263b052e0fd940f6d32).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367753200
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
 
 Review comment:
   yea, it looks ok.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367741733
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
 
 Review comment:
   Can you describe more here? Its like `Creates new references to avoid collisions between distinct and regular aggregate 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369335821
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,64 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) as cat2_cnt,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat2),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-576645591
 
 
   **[Test build #117179 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117179/testReport)** for PR 27058 at commit [`7a6b019`](https://github.com/apache/spark/commit/7a6b0197c1f32f6ad0d843b688b8abcc97344935).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579037098
 
 
   **[Test build #117456 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117456/testReport)** for PR 27058 at commit [`5c80418`](https://github.com/apache/spark/commit/5c804182b498c0e804cb0c3157d1dbd31d182e24).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572386876
 
 
   **[Test build #116345 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116345/testReport)** for PR 27058 at commit [`bcca92d`](https://github.com/apache/spark/commit/bcca92d14d13531bd02ab9f9eb19afe001f8754e).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575507740
 
 
   **[Test build #116911 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116911/testReport)** for PR 27058 at commit [`52953ec`](https://github.com/apache/spark/commit/52953ec6b3dec4dd6d48105df2f1ef889cd9e75d).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579038043
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575519274
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367717874
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -331,6 +457,17 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     }
   }
 
+  /**
+   * Collect all aggregate expressions.
 
 Review comment:
   We don't need this comment (it is totally the same with the function name below...

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367736708
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
 
 Review comment:
   nit: `filter(_.children.exists(!_.foldable))`

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-581008791
 
 
   Build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575466319
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572874663
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116434/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575499022
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508203
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21680/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575508247
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116911/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575519987
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578625013
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578625016
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22193/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578629266
 
 
   **[Test build #117434 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/117434/testReport)** for PR 27058 at commit [`7a74aae`](https://github.com/apache/spark/commit/7a74aae09f8f696102c5b92b850d572d64fd9cb1).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572874661
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575520172
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116895/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572821261
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21222/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367721603
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
 
 Review comment:
   If `expandAggregate` already has an expand (added by `expandDistinctAggregateWithFilter`) in a child, can we merge them inside `rewriteDistinctAggregate` (line 424)?
   https://github.com/apache/spark/pull/27058/files#r366138753

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367722603
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
 
 Review comment:
   nit:
   ```
     private def mayNeedtoRewrite(agg: Aggregate): Boolean = {
       val distinctAggs = collectAggregateExprs(agg).filter(_.isDistinct)
      ...
   ```

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573077769
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r366149894
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,6 +118,52 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat1_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat1_cnt2, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
 
 Review comment:
   Because I want to reuse `rewrite`. The first `Expand` created by `rewrite`.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575500747
 
 
   **[Test build #116907 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116907/testReport)** for PR 27058 at commit [`0f69488`](https://github.com/apache/spark/commit/0f69488539a4161c5605ce592b683b753aea0f1b).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575587787
 
 
   **[Test build #116939 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116939/testReport)** for PR 27058 at commit [`5462c0c`](https://github.com/apache/spark/commit/5462c0ce605cf95e608c945e4ca8bff358740f50).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579088588
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117456/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577579811
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-569911830
 
 
   **[Test build #115987 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115987/testReport)** for PR 27058 at commit [`0008bae`](https://github.com/apache/spark/commit/0008bae5ff27a6766b9ceb3bb067eac2e8e322dc).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368940820
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
 
 Review comment:
   So I should add filter API in dsl?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570140108
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572446217
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-569912064
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575498651
 
 
   **[Test build #116906 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116906/testReport)** for PR 27058 at commit [`bdda81d`](https://github.com/apache/spark/commit/bdda81dbaceb0ac7d80ebe0f03dbf79623fc6e42).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368849887
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,66 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat1_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat1_cnt2, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat1, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule serves two purposes:
+ * 1. Expand distinct aggregates which exists filter clause.
 
 Review comment:
   OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575519987
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368989206
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
 
 Review comment:
   Yea, I think its ok to do so.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575466319
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367723884
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularOperatorMap = regularAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+          val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+          val filterOpt = filter.map(_.transform {
+            case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+          })
+          val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+          (ae, aggExpr)
+      }
+
+      // Setup expand for the distinct aggregate expressions.
+      val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val rewriteDistinctOperatorMap = distinctAggExprs.map {
+        case ae @ AggregateExpression(af, _, _, filter, _) =>
+          // Why do we need to construct the phantom id ?
+          // First, In order to reduce costs, it is better to handle the filter clause locally.
+          // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+          // If(id > 1) 'a else null first, and use the result as output.
+          // Second, If more than one DISTINCT aggregate expression uses the same column,
+          // We need to construct the phantom attributes so as the output not lost.
+          // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+          // attribute '_gen_distinct-1 and attribute '_gen_distinct-2 instead of two 'a.
+          // Note: We just need to illusion the expression with filter clause.
+          // The illusionary mechanism may result in multiple distinct aggregations uses
+          // different column, so we still need to call `rewrite`.
 
 Review comment:
   Since this comment is stale, can you update/simplify it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367755294
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573140754
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574784598
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367741302
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
     case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
   }
 
   def rewrite(a: Aggregate): Aggregate = {
+    val expandAggregate = expandDistinctAggregateWithFilter(a)
+    rewriteDistinctAggregate(expandAggregate)
+  }
 
-    // Collect all aggregate expressions.
-    val aggExpressions = a.aggregateExpressions.flatMap { e =>
-      e.collect {
-        case ae: AggregateExpression => ae
+  private def expandDistinctAggregateWithFilter(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions.
+      val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
 
 Review comment:
   `val regularAggChildren = regularAggExpressions.flatMap(_.references)`?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578620873
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/22191/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368860880
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
+    val aggExpressions = collectAggregateExprs(a)
+    val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+    if (distinctAggExpressions.exists(_.filter.isDefined)) {
+      // Setup expand for the 'regular' aggregate expressions. Because we will construct a new
+      // aggregate, the children of the distinct aggregates will be changed to the generate
+      // ones, so we need creates new references to avoid collisions between distinct and
+      // regular aggregate children.
+      val regularAggExprs = regularAggExpressions.filter(_.children.exists(!_.foldable))
+      val regularFunChildren = regularAggExprs
+        .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+      val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+      val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+      val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+      val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+      val regularAggMap = regularAggExprs.map {
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r365092774
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
 
 Review comment:
   You can reference the discussion
   https://github.com/apache/spark/pull/26656#discussion_r353717031
   In order to reduce cost, we treat the filter conditions of DISTINCT aggregate expr in first aggregate or local is better.
   If we use `AggregationIterator`, the filter conditions of DISTINCT aggregate expr will be treated in second or thrid 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364515132
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -317,7 +363,74 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
     } else {
-      a
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            val phantomId = i + 1
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"phantom$phantomId-${e.sql}", e.dataType, nullable = true)())
+            }
+            val exprAttrLookup = exprAttrs.toMap
+            val newChildren = af.children.map(c => exprAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = None)
+            // Expand projection
+            val projection = unfoldableChildren.map {
+              case e if filter.isDefined => If(filter.get, e, nullify(e))
+              case e => e
+            }
+            (projection, exprAttrs, (ae, aggExpr))
+        }
+        val rewriteDistinctAttrMap = rewriteDistinctOperatorMap.map(_._2).flatten
+        val distinctAggChildAttrs = rewriteDistinctAttrMap.map(_._2)
+        val allAggAttrs = (regularAggChildAttrMap.map(_._2) ++ distinctAggChildAttrs)
 
 Review comment:
   nit: you don't need `(` and `)`

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574784613
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116781/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577581549
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367736428
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -151,21 +197,101 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     // We need at least two distinct aggregates for this rule because aggregation
     // strategy can handle a single distinct group.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1
 
 Review comment:
   Great

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578751948
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578876419
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-579220859
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117472/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572864758
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21233/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367758175
 
 

 ##########
 File path: sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out
 ##########
 @@ -444,10 +444,29 @@ NULL	1
 
 
 -- !query 34
-select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col)
+select ten, sum(distinct four) filter (where string(four) like '123') from onek a
+group by rollup(ten)
 -- !query 34 schema
-struct<count(1):bigint>
+struct<ten:int,sum(DISTINCT four) FILTER (WHERE CAST(four AS STRING) LIKE 123):bigint>
 -- !query 34 output
 
 Review comment:
   The same

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574963615
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116803/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573040278
 
 
   **[Test build #116490 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116490/testReport)** for PR 27058 at commit [`b425eb4`](https://github.com/apache/spark/commit/b425eb4eefecff3135af9bdc7386d9dfe109b1b1).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575675292
 
 
   **[Test build #116939 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116939/testReport)** for PR 27058 at commit [`5462c0c`](https://github.com/apache/spark/commit/5462c0ce605cf95e608c945e4ca8bff358740f50).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-577670237
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/117291/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer closed pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer closed pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058
 
 
   

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369306575
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +205,102 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // We need at least two distinct aggregates or a single distinct aggregate with a filter for
+    // this rule because aggregation strategy can handle a single distinct group without a filter.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
 
 Review comment:
   not `distinctAggs.size >= 1` but `distinctAggs.size > 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369341486
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,64 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) as cat2_cnt,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat2),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat2, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule consists of the two phases as follows.
+ *
+ * In the first phase, expands distinct aggregates which exists filter clause:
+ * 1. Guaranteed to compute filter clauses in the first aggregate locally.
+ * 2. The attributes referenced by different distinct aggregate expressions are likely to overlap,
+ *    and if no additional processing is performed, data loss will occur. To prevent this, we
+ *    generate new attributes and replace the original ones.
+ * 3. If we apply the first phase to distinct aggregate expressions which exists filter
+ *    clause, the aggregate after expand may have at least two distinct aggregates, so we need to
+ *    apply the second phase too. Please refer to the second phase for more details.
+ *
+ * In the second phase, rewrite when aggregate exists at least two distinct aggregates:
  * 1. Expand the data. There are three aggregation groups in this query:
 
 Review comment:
   The second example is not related to `RewriteDistinctAggregates`.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r371574438
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
 ##########
 @@ -165,20 +165,35 @@ package object dsl {
     }
 
     def rand(e: Long): Expression = Rand(e)
-    def sum(e: Expression): Expression = Sum(e).toAggregateExpression()
-    def sumDistinct(e: Expression): Expression = Sum(e).toAggregateExpression(isDistinct = true)
-    def count(e: Expression): Expression = Count(e).toAggregateExpression()
+    def sum(e: Expression, filter: Option[Expression] = None): Expression =
+      Sum(e).toAggregateExpression(false, filter)
+    def sumDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Sum(e).toAggregateExpression(true, filter)
+    def count(e: Expression, filter: Option[Expression] = None): Expression =
+      Count(e).toAggregateExpression(false, filter)
     def countDistinct(e: Expression*): Expression =
       Count(e).toAggregateExpression(isDistinct = true)
-    def approxCountDistinct(e: Expression, rsd: Double = 0.05): Expression =
-      HyperLogLogPlusPlus(e, rsd).toAggregateExpression()
-    def avg(e: Expression): Expression = Average(e).toAggregateExpression()
-    def first(e: Expression): Expression = new First(e).toAggregateExpression()
-    def last(e: Expression): Expression = new Last(e).toAggregateExpression()
-    def min(e: Expression): Expression = Min(e).toAggregateExpression()
-    def minDistinct(e: Expression): Expression = Min(e).toAggregateExpression(isDistinct = true)
-    def max(e: Expression): Expression = Max(e).toAggregateExpression()
-    def maxDistinct(e: Expression): Expression = Max(e).toAggregateExpression(isDistinct = true)
+    def countDistinct(filter: Option[Expression], e: Expression*): Expression =
+      Count(e).toAggregateExpression(true, filter)
+    def approxCountDistinct(
+        e: Expression,
+        rsd: Double = 0.05,
+        filter: Option[Expression] = None): Expression =
+      HyperLogLogPlusPlus(e, rsd).toAggregateExpression(true, filter)
+    def avg(e: Expression, filter: Option[Expression] = None): Expression =
+      Average(e).toAggregateExpression(false, filter)
+    def first(e: Expression, filter: Option[Expression] = None): Expression =
+      new First(e).toAggregateExpression(false, filter)
+    def last(e: Expression, filter: Option[Expression] = None): Expression =
+      new Last(e).toAggregateExpression(false, filter)
+    def min(e: Expression, filter: Option[Expression] = None): Expression =
+      Min(e).toAggregateExpression(false, filter)
+    def minDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Min(e).toAggregateExpression(true, filter)
+    def max(e: Expression, filter: Option[Expression] = None): Expression =
+      Max(e).toAggregateExpression(false, filter)
+    def maxDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Max(e).toAggregateExpression(true, filter)
     def upper(e: Expression): Expression = Upper(e)
 
 Review comment:
   These changes above are related to `RewriteDistinctAggregatesSuite`? If no, we don't need to change this file.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578625013
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r367758738
 
 

 ##########
 File path: sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out
 ##########
 @@ -95,41 +95,73 @@ struct<count(id) FILTER (WHERE (date_format(CAST(hiredate AS TIMESTAMP), yyyy-MM
 
 
 -- !query 9
-SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a
+SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp
 -- !query 9 schema
-struct<a:int,count(b) FILTER (WHERE (a >= 2)):bigint>
+struct<count(DISTINCT id) FILTER (WHERE (date_format(CAST(hiredate AS TIMESTAMP), yyyy-MM-dd HH:mm:ss) = 2001-01-01 00:00:00)):bigint>
 -- !query 9 output
+2
 
 Review comment:
   I can't visit this url.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364236488
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
 ##########
 @@ -192,8 +192,10 @@ class AnalysisErrorSuite extends AnalysisTest {
 
   errorTest(
     "DISTINCT and FILTER cannot be used in aggregate functions at the same time",
-    CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"),
-    "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil)
+    CatalystSqlParser.parsePlan("SELECT sum(DISTINCT b), " +
+      "count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"),
+    "When there are multiple DISTINCT aggregate expressions acting on different " +
+    "fields, any DISTINCT aggregate expression not allow use FILTER clause." :: Nil)
 
 Review comment:
   Can you add some tests for the foldable case?

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r366155166
 
 

 ##########
 File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
 ##########
 @@ -192,8 +192,10 @@ class AnalysisErrorSuite extends AnalysisTest {
 
   errorTest(
     "DISTINCT and FILTER cannot be used in aggregate functions at the same time",
-    CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"),
-    "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil)
+    CatalystSqlParser.parsePlan("SELECT sum(DISTINCT b), " +
+      "count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"),
+    "When there are multiple DISTINCT aggregate expressions acting on different " +
+    "fields, any DISTINCT aggregate expression not allow use FILTER clause." :: Nil)
 
 Review comment:
   `emp_name` is foldable? I meant a query like `select key, sum(distinct value), avg(distinct 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364537516
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,6 +118,52 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat2_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) 'phantom1 else null),
 
 Review comment:
   You can reference https://github.com/apache/spark/blob/a4fd143fa92cb569764b669e5c53f6dfd1f1ae6a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala#L336
   

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574620588
 
 
   **[Test build #116781 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116781/testReport)** for PR 27058 at commit [`aa37870`](https://github.com/apache/spark/commit/aa37870ec36a8bf182c360a37c46d138d267f8a4).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r371574438
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
 ##########
 @@ -165,20 +165,35 @@ package object dsl {
     }
 
     def rand(e: Long): Expression = Rand(e)
-    def sum(e: Expression): Expression = Sum(e).toAggregateExpression()
-    def sumDistinct(e: Expression): Expression = Sum(e).toAggregateExpression(isDistinct = true)
-    def count(e: Expression): Expression = Count(e).toAggregateExpression()
+    def sum(e: Expression, filter: Option[Expression] = None): Expression =
+      Sum(e).toAggregateExpression(false, filter)
+    def sumDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Sum(e).toAggregateExpression(true, filter)
+    def count(e: Expression, filter: Option[Expression] = None): Expression =
+      Count(e).toAggregateExpression(false, filter)
     def countDistinct(e: Expression*): Expression =
       Count(e).toAggregateExpression(isDistinct = true)
-    def approxCountDistinct(e: Expression, rsd: Double = 0.05): Expression =
-      HyperLogLogPlusPlus(e, rsd).toAggregateExpression()
-    def avg(e: Expression): Expression = Average(e).toAggregateExpression()
-    def first(e: Expression): Expression = new First(e).toAggregateExpression()
-    def last(e: Expression): Expression = new Last(e).toAggregateExpression()
-    def min(e: Expression): Expression = Min(e).toAggregateExpression()
-    def minDistinct(e: Expression): Expression = Min(e).toAggregateExpression(isDistinct = true)
-    def max(e: Expression): Expression = Max(e).toAggregateExpression()
-    def maxDistinct(e: Expression): Expression = Max(e).toAggregateExpression(isDistinct = true)
+    def countDistinct(filter: Option[Expression], e: Expression*): Expression =
+      Count(e).toAggregateExpression(true, filter)
+    def approxCountDistinct(
+        e: Expression,
+        rsd: Double = 0.05,
+        filter: Option[Expression] = None): Expression =
+      HyperLogLogPlusPlus(e, rsd).toAggregateExpression(true, filter)
+    def avg(e: Expression, filter: Option[Expression] = None): Expression =
+      Average(e).toAggregateExpression(false, filter)
+    def first(e: Expression, filter: Option[Expression] = None): Expression =
+      new First(e).toAggregateExpression(false, filter)
+    def last(e: Expression, filter: Option[Expression] = None): Expression =
+      new Last(e).toAggregateExpression(false, filter)
+    def min(e: Expression, filter: Option[Expression] = None): Expression =
+      Min(e).toAggregateExpression(false, filter)
+    def minDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Min(e).toAggregateExpression(true, filter)
+    def max(e: Expression, filter: Option[Expression] = None): Expression =
+      Max(e).toAggregateExpression(false, filter)
+    def maxDistinct(e: Expression, filter: Option[Expression] = None): Expression =
+      Max(e).toAggregateExpression(true, filter)
     def upper(e: Expression): Expression = Upper(e)
 
 Review comment:
   These changes above are related to `RewriteDistinctAggregatesSuite`? If no, we don't need to change the file in this pr.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573140773
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116491/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r369336741
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,64 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) as cat2_cnt,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat2),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat2, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule consists of the two phases as follows.
+ *
+ * In the first phase, expands distinct aggregates which exists filter clause:
 
 Review comment:
   Sorry! I get it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-575520166
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368849887
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,7 +118,66 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
- * The rule does the following things here:
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat1_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) '_gen_distinct_1 else null),
+ *                   count(if (('gid = 2)) '_gen_distinct_2 else null),
+ *                   first(if (('gid = 0)) 'total else null) ignore nulls]
+ *      output = ['key, 'cat1_cnt, 'cat1_cnt2, 'total])
+ *     Aggregate(
+ *        key = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid]
+ *        functions = [sum('value)]
+ *        output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'total])
+ *       Expand(
+ *           projections = [('key, null, null, 0, 'value),
+ *                         ('key, '_gen_distinct_1, null, 1, null),
+ *                         ('key, null, '_gen_distinct_2, 2, null)]
+ *           output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'gid, 'value])
+ *         Expand(
+ *            projections = [('key, if ('id > 1) 'cat1 else null, 'cat1, cast('value as bigint))]
+ *            output = ['key, '_gen_distinct_1, '_gen_distinct_2, 'value])
+ *           LocalTableScan [...]
+ * }}}
+ *
+ * The rule serves two purposes:
+ * 1. Expand distinct aggregates which exists filter clause.
 
 Review comment:
   OK. Looks better.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-572864514
 
 
   **[Test build #116445 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116445/testReport)** for PR 27058 at commit [`dd09645`](https://github.com/apache/spark/commit/dd096453c640335fe7af9f8b89eac296ee54615c).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r364537771
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -118,6 +118,52 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: single distinct aggregate function with filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt1,
+ *     COUNT(DISTINCT cat1) as cat1_cnt2,
+ *     SUM(value) AS total
+ *  FROM
+ *    data
+ *  GROUP BY
+ *    key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) with FILTER('id > 1),
+ *                 COUNT(DISTINCT 'cat1),
+ *                 sum('value)]
+ *    output = ['key, 'cat1_cnt1, 'cat2_cnt2, 'total])
+ *   LocalTableScan [...]
+ * }}}
+ *
+ * This rule rewrites this logical plan to the following (pseudo) logical plan:
+ * {{{
+ *   Aggregate(
+ *      key = ['key]
+ *      functions = [count(if (('gid = 1)) 'phantom1 else null),
 
 Review comment:
   Yea, I got it now. 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570140195
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20809/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-574784598
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#discussion_r368859956
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -148,24 +207,106 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
     val distinctAggs = exprs.flatMap { _.collect {
       case ae: AggregateExpression if ae.isDistinct => ae
     }}
-    // We need at least two distinct aggregates for this rule because aggregation
-    // strategy can handle a single distinct group.
+    // This rule serves two purposes:
+    // One is to rewrite when there exists at least two distinct aggregates. We need at least
+    // two distinct aggregates for this rule because aggregation strategy can handle a single
+    // distinct group.
+    // Another is to expand distinct aggregates which exists filter clause so that we can
+    // evaluate the filter locally.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size >= 1 || distinctAggs.exists(_.filter.isDefined)
   }
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) => rewrite(a)
+    case a: Aggregate if mayNeedtoRewrite(a.aggregateExpressions) =>
+      val expandAggregate = extractFiltersInDistinctAggregate(a)
+      rewriteDistinctAggregate(expandAggregate)
   }
 
-  def rewrite(a: Aggregate): Aggregate = {
+  private def extractFiltersInDistinctAggregate(a: Aggregate): Aggregate = {
 
 Review comment:
   For first suggestion, you means I should add new API in dsl? dsl can't support filter clause yet.
   For second suggestion, OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-573040878
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21277/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r365107970
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -316,6 +362,86 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }.asInstanceOf[NamedExpression]
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
+    } else if (distinctAggGroups.size == 1) {
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            // Why do we need to construct the phantom id ?
+            // First, In order to reduce costs, it is better to handle the filter clause locally.
+            // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+            // If(id > 1) 'a else null first, and use the result as output.
+            // Second, If more than one DISTINCT aggregate expression uses the same column,
+            // We need to construct the phantom attributes so as the output not lost.
+            // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+            // attribute 'phantom1-a and attribute 'phantom2-a instead of two 'a.
+            // Note: We just need to illusion the expression with filter clause.
+            // The illusionary mechanism may result in multiple distinct aggregations uses
+            // different column, so we still need to call `rewrite`.
+            val phantomId = i + 1
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"phantom$phantomId-${e.sql}", e.dataType, nullable = true)())
 
 Review comment:
   OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570139949
 
 
   **[Test build #116017 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116017/testReport)** for PR 27058 at commit [`a4fd143`](https://github.com/apache/spark/commit/a4fd143fa92cb569764b669e5c53f6dfd1f1ae6a).

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#issuecomment-570184651
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27058: [SPARK-30395][SQL] When one or more DISTINCT aggregate expressions operate on the same field, the DISTINCT aggregate expression allows the use of the FILTER clause
URL: https://github.com/apache/spark/pull/27058#discussion_r365112646
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
 ##########
 @@ -316,6 +362,86 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }.asInstanceOf[NamedExpression]
       }
       Aggregate(groupByAttrs, patchedAggExpressions, firstAggregate)
+    } else if (distinctAggGroups.size == 1) {
+      val (distinctAggExpressions, regularAggExpressions) = aggExpressions.partition(_.isDistinct)
+      if (distinctAggExpressions.exists(_.filter.isDefined)) {
+        val regularAggExprs = regularAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val regularFunChildren = regularAggExprs
+          .flatMap(_.aggregateFunction.children.filter(!_.foldable))
+        val regularFilterAttrs = regularAggExprs.flatMap(_.filterAttributes)
+        val regularAggChildren = (regularFunChildren ++ regularFilterAttrs).distinct
+        val regularAggChildAttrMap = regularAggChildren.map(expressionAttributePair)
+        val regularAggChildAttrLookup = regularAggChildAttrMap.toMap
+        val regularOperatorMap = regularAggExprs.map {
+          case ae @ AggregateExpression(af, _, _, filter, _) =>
+            val newChildren = af.children.map(c => regularAggChildAttrLookup.getOrElse(c, c))
+            val raf = af.withNewChildren(newChildren).asInstanceOf[AggregateFunction]
+            val filterOpt = filter.map(_.transform {
+              case a: Attribute => regularAggChildAttrLookup.getOrElse(a, a)
+            })
+            val aggExpr = ae.copy(aggregateFunction = raf, filter = filterOpt)
+            (ae, aggExpr)
+        }
+        val distinctAggExprs = distinctAggExpressions.filter(e => e.children.exists(!_.foldable))
+        val rewriteDistinctOperatorMap = distinctAggExprs.zipWithIndex.map {
+          case (ae @ AggregateExpression(af, _, _, filter, _), i) =>
+            // Why do we need to construct the phantom id ?
+            // First, In order to reduce costs, it is better to handle the filter clause locally.
+            // e.g. COUNT (DISTINCT a) FILTER (WHERE id > 1), evaluate expression
+            // If(id > 1) 'a else null first, and use the result as output.
+            // Second, If more than one DISTINCT aggregate expression uses the same column,
+            // We need to construct the phantom attributes so as the output not lost.
+            // e.g. SUM (DISTINCT a), COUNT (DISTINCT a) FILTER (WHERE id > 1) will output
+            // attribute 'phantom1-a and attribute 'phantom2-a instead of two 'a.
+            // Note: We just need to illusion the expression with filter clause.
+            // The illusionary mechanism may result in multiple distinct aggregations uses
+            // different column, so we still need to call `rewrite`.
+            val phantomId = i + 1
+            val unfoldableChildren = af.children.filter(!_.foldable)
+            val exprAttrs = unfoldableChildren.map { e =>
+              (e, AttributeReference(s"phantom$phantomId-${e.sql}", e.dataType, nullable = true)())
 
 Review comment:
   Since this name is shown in explain, I think a related name is better. Why `phantom`? I think common terms used in spark are better.

----------------------------------------------------------------
This is an automated message from the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27058: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
URL: https://github.com/apache/spark/pull/27058#issuecomment-578699422
 
 
   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


With regards,
Apache Git Services

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