You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/07/29 17:41:43 UTC

[GitHub] [spark] beliefer opened a new pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   ### 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 which 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;
   ```
   
   ### Why are the changes needed?
   Spark SQL only support use FILTER clause on aggregate expression without DISTINCT.
   This PR support Filter expression allows simultaneous use of DISTINCT
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes
   
   
   ### How was this patch tested?
   Exists and 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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463454895



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +257,30 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))
+      val (distinctAggFilters, distinctAggFilterAttrs, maxConds) = distinctAggExprs.collect {
+        case AggregateExpression(_, _, _, filter, _) if filter.isDefined =>
+          val (e, attr) = expressionAttributePair(filter.get)
+          val aggregateExp = AggregateExpression(Max(attr), Partial, false)

Review comment:
       Good!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463448075



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -118,6 +118,49 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: aggregate function with distinct and filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) FILTER (WHERE id > 2) as cat2_cnt,
+ *     SUM(value) FILTER (WHERE id > 3) AS total
+ *   FROM
+ *     data
+ *   GROUP BY
+ *     key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) FILTER (WHERE 'id > 1),
+ *                 COUNT(DISTINCT 'cat2) FILTER (WHERE 'id > 2),
+ *                 sum('value) FILTER (WHERE 'id > 3)]
+ *    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) and 'max_cond1) 'cat1 else null),

Review comment:
       can we add a comment here so that it's easier for others to understand?
   ```
   // If 'max_cond1 is true, it means at least one row of a distinct value satisfies the filter and should be
   // included in the aggregate function.
   functions = [count(if (('gid = 1) and 'max_cond1) 'cat1 else null),
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126857 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126857/testReport)** for PR 29291 at commit [`fbb051b`](https://github.com/apache/spark/commit/fbb051bd574d2bce7e8794fce76ebe51dcc28cb3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126886 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126886/testReport)** for PR 29291 at commit [`abafc20`](https://github.com/apache/spark/commit/abafc208acb8fa6f2987a2ee00b1761a26086266).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463395906



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +257,30 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))

Review comment:
       I think we should just use `distinctAggs` here. We can add test for this 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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #29291:
URL: https://github.com/apache/spark/pull/29291


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463043472



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -184,8 +227,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }
     }
 
-    // Aggregation strategy can handle queries with a single distinct group.
-    if (distinctAggGroups.size > 1) {

Review comment:
       Let's use `If (distinctAggGroups.size > 1 || distinctAggs.exists(_.filter.isDefined))` 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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126956 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126956/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] cloud-fan commented on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29291:
URL: https://github.com/apache/spark/pull/29291#issuecomment-668105911


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126797 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126797/testReport)** for PR 29291 at commit [`145a9dd`](https://github.com/apache/spark/commit/145a9dd2b315f25047d7032b6027a74bef34d202).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463441087



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +257,30 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))

Review comment:
       ah i see




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463448075



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -118,6 +118,49 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: aggregate function with distinct and filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) FILTER (WHERE id > 2) as cat2_cnt,
+ *     SUM(value) FILTER (WHERE id > 3) AS total
+ *   FROM
+ *     data
+ *   GROUP BY
+ *     key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) FILTER (WHERE 'id > 1),
+ *                 COUNT(DISTINCT 'cat2) FILTER (WHERE 'id > 2),
+ *                 sum('value) FILTER (WHERE 'id > 3)]
+ *    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) and 'max_cond1) 'cat1 else null),

Review comment:
       can we add a comment here so that it's easier for others to understand?
   ```
   // If 'max_cond1 is true, it means at least one row of a distinct value satisfies the filter.
   // This distinct value should be included in the aggregate function.
   functions = [count(if (('gid = 1) and 'max_cond1) 'cat1 else null),
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] cloud-fan commented on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29291:
URL: https://github.com/apache/spark/pull/29291#issuecomment-667990596


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126994/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462957297



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +256,35 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))
+      val distinctAggFilterAttrMap = distinctAggExprs.collect {

Review comment:
       this is `distinctAggFilters.zip(maxCond.map(_.toAttribute)).toMap`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126953 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126953/testReport)** for PR 29291 at commit [`39583dd`](https://github.com/apache/spark/commit/39583dde43da9580245cd34768d3f613fab8b090).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] cloud-fan commented on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29291:
URL: https://github.com/apache/spark/pull/29291#issuecomment-666892983


   can you rebase/merge with the master branch to get the github action fix? The jenkin is quite unstable now and we may need to rely on github actions


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463395762



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +257,30 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))

Review comment:
       how about `count(distinct 1) FILTER (WHERE ...)`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463016625



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +256,35 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))
+      val distinctAggFilterAttrMap = distinctAggExprs.collect {

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126968 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126968/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126994/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126857 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126857/testReport)** for PR 29291 at commit [`fbb051b`](https://github.com/apache/spark/commit/fbb051bd574d2bce7e8794fce76ebe51dcc28cb3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126778 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126778/testReport)** for PR 29291 at commit [`4ba808b`](https://github.com/apache/spark/commit/4ba808bf602d39330810ce0a3bc61fe2ba9ef2b5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462924626



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -144,28 +192,23 @@ import org.apache.spark.sql.types.IntegerType
  */
 object RewriteDistinctAggregates extends Rule[LogicalPlan] {
 
-  private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = {
-    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.
+  private def mayNeedtoRewrite(a: Aggregate): Boolean = {
+    val aggExpressions = collectAggregateExprs(a)
+    val distinctAggs = aggExpressions.filter(_.isDistinct)
+    // We need at least two distinct aggregates or the single distinct aggregate group exists filter
+    // clause for this rule because aggregation strategy can handle a single distinct aggregate
+    // group without filter clause.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size > 1 || (distinctAggs.size == 1 && aggExpressions.exists(_.filter.isDefined))

Review comment:
       If `distinctAggs.size == 0` and `aggExpressions.exists(_.filter.isDefined)`,  we not need this 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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463454135



##########
File path: sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql
##########
@@ -36,8 +36,11 @@ SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp;
 SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp;
 SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp;
 SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp;
--- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT
--- SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp;
+SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp;
+SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp;
+SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp;
+SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp;
+SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData;

Review comment:
       can we also test `COUNT(DISTINCT id) FILTER (WHERE true)` and `COUNT(DISTINCT id) FILTER (WHERE false)`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126866 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126866/testReport)** for PR 29291 at commit [`9939ea7`](https://github.com/apache/spark/commit/9939ea7a852685a3d31136e30e2a9e44fbe7fb5f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463449645



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +257,30 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))
+      val (distinctAggFilters, distinctAggFilterAttrs, maxConds) = distinctAggExprs.collect {
+        case AggregateExpression(_, _, _, filter, _) if filter.isDefined =>
+          val (e, attr) = expressionAttributePair(filter.get)
+          val aggregateExp = AggregateExpression(Max(attr), Partial, false)

Review comment:
       nit: `Max(attr).toAggregateExpression(distinct = false)`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126843 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126843/testReport)** for PR 29291 at commit [`7362dfb`](https://github.com/apache/spark/commit/7362dfbdc165f0ff24e24ac02074601852447cf8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126980 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126980/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   @cloud-fan Thanks for your review and good idea.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463421409



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +257,30 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))

Review comment:
       Because filter is child of agg expression too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126843 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126843/testReport)** for PR 29291 at commit [`7362dfb`](https://github.com/apache/spark/commit/7362dfbdc165f0ff24e24ac02074601852447cf8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   > can you rebase/merge with the master branch to get the github action fix? The jenkin is quite unstable now and we may need to rely on github actions
   
   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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126866 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126866/testReport)** for PR 29291 at commit [`9939ea7`](https://github.com/apache/spark/commit/9939ea7a852685a3d31136e30e2a9e44fbe7fb5f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   **[Test build #126810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126810/testReport)** for PR 29291 at commit [`7362dfb`](https://github.com/apache/spark/commit/7362dfbdc165f0ff24e24ac02074601852447cf8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126980 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126980/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


   cc @cloud-fan 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] cloud-fan commented on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29291:
URL: https://github.com/apache/spark/pull/29291#issuecomment-667865844


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463018648



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -144,28 +192,23 @@ import org.apache.spark.sql.types.IntegerType
  */
 object RewriteDistinctAggregates extends Rule[LogicalPlan] {
 
-  private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = {
-    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.
+  private def mayNeedtoRewrite(a: Aggregate): Boolean = {
+    val aggExpressions = collectAggregateExprs(a)
+    val distinctAggs = aggExpressions.filter(_.isDistinct)
+    // We need at least two distinct aggregates or the single distinct aggregate group exists filter
+    // clause for this rule because aggregation strategy can handle a single distinct aggregate
+    // group without filter clause.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size > 1 || (distinctAggs.size == 1 && aggExpressions.exists(_.filter.isDefined))

Review comment:
       OK. I got 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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126886 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126886/testReport)** for PR 29291 at commit [`abafc20`](https://github.com/apache/spark/commit/abafc208acb8fa6f2987a2ee00b1761a26086266).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462957711



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -144,28 +192,23 @@ import org.apache.spark.sql.types.IntegerType
  */
 object RewriteDistinctAggregates extends Rule[LogicalPlan] {
 
-  private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = {
-    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.
+  private def mayNeedtoRewrite(a: Aggregate): Boolean = {
+    val aggExpressions = collectAggregateExprs(a)
+    val distinctAggs = aggExpressions.filter(_.isDistinct)
+    // We need at least two distinct aggregates or the single distinct aggregate group exists filter
+    // clause for this rule because aggregation strategy can handle a single distinct aggregate
+    // group without filter clause.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size > 1 || (distinctAggs.size == 1 && aggExpressions.exists(_.filter.isDefined))

Review comment:
       shouldn't it be `distinctAggs.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



---------------------------------------------------------------------
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 #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463549738



##########
File path: sql/core/src/test/resources/sql-tests/inputs/group-by-filter.sql
##########
@@ -36,8 +36,11 @@ SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp;
 SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp;
 SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp;
 SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp;
--- [SPARK-30276] Support Filter expression allows simultaneous use of DISTINCT
--- SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp;
+SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp;
+SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp;
+SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp;
+SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp;
+SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData;

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462958983



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -184,8 +227,8 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
         }
     }
 
-    // Aggregation strategy can handle queries with a single distinct group.
-    if (distinctAggGroups.size > 1) {

Review comment:
       For `count(distinct c1), count(c2) FILTER ...`, we don't apply this rule previously and it works well. Why do we need to apply this rule 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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] cloud-fan commented on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29291:
URL: https://github.com/apache/spark/pull/29291#issuecomment-668375534


   thanks, merging to master!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r463545124



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -118,6 +118,49 @@ import org.apache.spark.sql.types.IntegerType
  *       LocalTableScan [...]
  * }}}
  *
+ * Third example: aggregate function with distinct and filter clauses (in sql):
+ * {{{
+ *   SELECT
+ *     COUNT(DISTINCT cat1) FILTER (WHERE id > 1) as cat1_cnt,
+ *     COUNT(DISTINCT cat2) FILTER (WHERE id > 2) as cat2_cnt,
+ *     SUM(value) FILTER (WHERE id > 3) AS total
+ *   FROM
+ *     data
+ *   GROUP BY
+ *     key
+ * }}}
+ *
+ * This translates to the following (pseudo) logical plan:
+ * {{{
+ * Aggregate(
+ *    key = ['key]
+ *    functions = [COUNT(DISTINCT 'cat1) FILTER (WHERE 'id > 1),
+ *                 COUNT(DISTINCT 'cat2) FILTER (WHERE 'id > 2),
+ *                 sum('value) FILTER (WHERE 'id > 3)]
+ *    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) and 'max_cond1) 'cat1 else 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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462924626



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -144,28 +192,23 @@ import org.apache.spark.sql.types.IntegerType
  */
 object RewriteDistinctAggregates extends Rule[LogicalPlan] {
 
-  private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = {
-    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.
+  private def mayNeedtoRewrite(a: Aggregate): Boolean = {
+    val aggExpressions = collectAggregateExprs(a)
+    val distinctAggs = aggExpressions.filter(_.isDistinct)
+    // We need at least two distinct aggregates or the single distinct aggregate group exists filter
+    // clause for this rule because aggregation strategy can handle a single distinct aggregate
+    // group without filter clause.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size > 1 || (distinctAggs.size == 1 && aggExpressions.exists(_.filter.isDefined))

Review comment:
       If `distinctAggs.size == 0` and `aggExpressions.exists(_.filter.isDefined)`,  we not need this rewrite.
   The normal agg with filter could treated by physical plan.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126810/testReport)** for PR 29291 at commit [`7362dfb`](https://github.com/apache/spark/commit/7362dfbdc165f0ff24e24ac02074601852447cf8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462958305



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -144,28 +192,23 @@ import org.apache.spark.sql.types.IntegerType
  */
 object RewriteDistinctAggregates extends Rule[LogicalPlan] {
 
-  private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = {
-    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.
+  private def mayNeedtoRewrite(a: Aggregate): Boolean = {
+    val aggExpressions = collectAggregateExprs(a)
+    val distinctAggs = aggExpressions.filter(_.isDistinct)
+    // We need at least two distinct aggregates or the single distinct aggregate group exists filter
+    // clause for this rule because aggregation strategy can handle a single distinct aggregate
+    // group without filter clause.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size > 1 || (distinctAggs.size == 1 && aggExpressions.exists(_.filter.isDefined))

Review comment:
       shall we match https://github.com/apache/spark/pull/29291/files#diff-29e82df7487a97f879691c1b525709aeR231 ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] beliefer commented on a change in pull request #29291: [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 #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462925756



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +256,35 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))
+      val distinctAggFilterAttrMap = distinctAggExprs.collect {

Review comment:
       But I want
   ```
   val distinctAggFilterAttrLookup = distinctAggFilterAttrMap.map { tuple3 =>
           tuple3._1 -> tuple3._3.toAttribute
         }.toMap
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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


   **[Test build #126968 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126968/testReport)** for PR 29291 at commit [`883973b`](https://github.com/apache/spark/commit/883973b9bc8a9c530a002cf4b48217546929fb5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462917670



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -207,13 +256,35 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
       val distinctAggChildren = distinctAggGroups.keySet.flatten.toSeq.distinct
       val distinctAggChildAttrMap = distinctAggChildren.map(expressionAttributePair)
       val distinctAggChildAttrs = distinctAggChildAttrMap.map(_._2)
+      // Setup all the filters in distinct aggregate.
+      val distinctAggExprs = aggExpressions
+        .filter(e => e.isDistinct && e.children.exists(!_.foldable))
+      val distinctAggFilterAttrMap = distinctAggExprs.collect {

Review comment:
       nit: `val (distinctAggFilters, distinctAggFilterAttrs, maxCond) = distinctAggExprs.collect(...).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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29291: [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29291:
URL: https://github.com/apache/spark/pull/29291#discussion_r462915233



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala
##########
@@ -144,28 +192,23 @@ import org.apache.spark.sql.types.IntegerType
  */
 object RewriteDistinctAggregates extends Rule[LogicalPlan] {
 
-  private def mayNeedtoRewrite(exprs: Seq[Expression]): Boolean = {
-    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.
+  private def mayNeedtoRewrite(a: Aggregate): Boolean = {
+    val aggExpressions = collectAggregateExprs(a)
+    val distinctAggs = aggExpressions.filter(_.isDistinct)
+    // We need at least two distinct aggregates or the single distinct aggregate group exists filter
+    // clause for this rule because aggregation strategy can handle a single distinct aggregate
+    // group without filter clause.
     // This check can produce false-positives, e.g., SUM(DISTINCT a) & COUNT(DISTINCT a).
-    distinctAggs.size > 1
+    distinctAggs.size > 1 || (distinctAggs.size == 1 && aggExpressions.exists(_.filter.isDefined))

Review comment:
       We can remove `distinctAggs.size == 1`, as it's indicarted by `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



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