You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "beliefer (via GitHub)" <gi...@apache.org> on 2023/07/29 08:02:11 UTC

[GitHub] [spark] beliefer opened a new pull request, #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

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

   ### What changes were proposed in this pull request?
   Some queries contains multiple scalar subquery(aggregation without group by clause) and connected with join.
   For example,
   ```
   SELECT *
   FROM (SELECT
     avg(power) avg_power,
     count(power) count_power,
     count(DISTINCT power) count_distinct_power
   FROM data
   WHERE country = "USA"
     AND (id BETWEEN 1 AND 3
     OR city = "Berkeley"
     OR name = "Xiao")) B1,
     (SELECT
       avg(power) avg_power,
       count(power) count_power,
       count(DISTINCT power) count_distinct_power
     FROM data
     WHERE country = "China"
       AND (id BETWEEN 4 AND 5
       OR city = "Hangzhou"
       OR name = "Wenchen")) B2
   ```
   If we can merge the filters and aggregates, we can scan data source only once and eliminate the join so as avoid shuffle.
   
   This PR also supports eliminate nested Join, please refer to: https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q28.sql
   
   Obviously, this change will improve the performance.
   
   
   ### Why are the changes needed?
   Improve the performance for the case show above.
   
   
   ### Does this PR introduce _any_ user-facing change?
   'No'.
   New feature.
   
   
   ### How was this patch tested?
   New test cases and micro benchmark.
   
   ```
   Java HotSpot(TM) 64-Bit Server VM 1.8.0_311-b11 on Mac OS X 10.16
   Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz
   Benchmark EliminateJoinByCombineAggregate:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
   ---------------------------------------------------------------------------------------------------------------------------------------
   step is 1000000, EliminateJoinByCombineAggregate: false            601            738         119         34.9          28.6       1.0X
   step is 1000000, EliminateJoinByCombineAggregate: true)            721            747          28         29.1          34.4       0.8X
   step is 100000, EliminateJoinByCombineAggregate: false             355            402          68         59.0          16.9       1.7X
   step is 100000, EliminateJoinByCombineAggregate: true)             214            222          10         98.0          10.2       2.8X
   step is 10000, EliminateJoinByCombineAggregate: false              321            371          26         65.4          15.3       1.9X
   step is 10000, EliminateJoinByCombineAggregate: true)              158            183          17        132.5           7.5       3.8X
   step is 1000, EliminateJoinByCombineAggregate: false               311            348          24         67.5          14.8       1.9X
   step is 1000, EliminateJoinByCombineAggregate: true)               145            161          12        144.2           6.9       4.1X
   step is 100, EliminateJoinByCombineAggregate: false                305            330          27         68.7          14.6       2.0X
   step is 100, EliminateJoinByCombineAggregate: true)                142            151           8        147.8           6.8       4.2X
   step is 10, EliminateJoinByCombineAggregate: false                 310            345          29         67.7          14.8       1.9X
   step is 10, EliminateJoinByCombineAggregate: true)                 139            144           6        150.4           6.7       4.3X
   step is 1, EliminateJoinByCombineAggregate: false                  304            318          12         69.0          14.5       2.0X
   step is 1, EliminateJoinByCombineAggregate: true)                  119            125           5        175.8           5.7       5.0X
   ```
   


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281430619


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map(mapAttributes(_, outputMap))
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            Seq(
+              (la.aggregateExpressions, filters.head),
+              (rightAggregateExprs, filters.last)
+            ).flatMap { case (aggregateExpressions, propagatedFilter) =>
+              aggregateExpressions.map { ne =>
+                ne.transform {
+                  case ae @ AggregateExpression(_, _, _, filterOpt, _) =>
+                    val newFilter = filterOpt.map { filter =>
+                      And(filter, propagatedFilter)

Review Comment:
   SGTM.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1285042542


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   That's ok, it is not the conditions (`f(c)`s) relation to each other what matters, but the filter columns set (`c1` ... `c10`) relation to aggregate column set (`a`). 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283905000


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```
   
   @peter-toth Put the filter condition into project list will inflate data and take shuffle more stressful.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1667104811

   > Although that part is useful as in some cases we don't need to use any heuristics to allow merging 
   > (https://github.com/apache/spark/pull/42223#issuecomment-1665208447), it makes my PR complex. Maybe we can add it > back later...
   
   I agree.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1278918942


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.

Review Comment:
   `must not exists filter` what does it mean? do you mean `contains`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {

Review Comment:
   I think transform up is better. For a multi-join query, we can eliminate the leaf joins first, and transform up to eliminate all joins if possible.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)

Review Comment:
   we don't need to handle Join here. If the rule runs bottom up, leaf joins become Aggregate and then we can match simple join again.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {

Review Comment:
   nit: `CombineJoinedAggregates`



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did scan 1-1 partitions, so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be beneficial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 1-1 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I alowed aggregate merge if original query scans match or differ only in data filters depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config. (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates [spark]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-2078379711

   We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
   If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280715671


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(

Review Comment:
   the return type is a bit complicated, let's add comment to explain



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279067309


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {
+      case j @ Join(_, _, joinType, None, _) if isSupportedJoinType(joinType) =>
+        val aggregates = ArrayBuffer.empty[Aggregate]
+        if (collectAggregate(j, aggregates)) {
+          var finalAggregate: Option[LogicalPlan] = None
+          for ((aggregate, i) <- aggregates.tail.zipWithIndex

Review Comment:
   Got it now thanks! Maybe we could try merging an aggregate with all the previous ones that couldn't be merged together.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281465743


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesHelper.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+/**
+ * The helper class used to merge scalar subqueries.
+ */
+trait MergeScalarSubqueriesHelper {
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.

Review Comment:
   Oh. The comment is not correct. Let's updated it.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281561980


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Please see `test("join two side are Aggregates without Filter")`.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280788640


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   I mean this case branch seems to behave incorrectly when you have adjacent filters in both plans.
   E.g. if you have:
   ```
   SELECT avg(a)
   FROM (
     SELECT * FROM t WHERE c1 = 1
   ) t
   WHERE c2 = 1
   ```
   and
   ```
   SELECT sum(b)
   FROM (
     SELECT * FROM t WHERE c1 = 2
   ) t
   WHERE c2 = 2
   ```
   and `PushDownPredicates` is disabled then you are in that situation. When you are merging the parent `Filter` nodes you shouldn`t ignore the propagated filters from child `Filter` nodes.
   
   Actually https://github.com/apache/spark/pull/37630 handles that case and many others and that's why I think splitting this PR into 2 parts would be better:
   - This rule would be still useful if you used the existing `MergeScalarSubqueries.tryMergePlans()` to merge plans. It couldn't deal with Q28 immediately, but could deal with simpler aggregates without differing filters.
   - https://github.com/apache/spark/pull/37630 could be a good base for implementing the filter merging feature. We could simplify that PR and remove the tricky physical plan checking part and introduce a simple boolean flag to enable/disable the filter merging feature like this PR does. In this case both Q9 and Q28 could benefit from the new feature...



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281825698


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   >  Now, if the conditions differ then it isn't trivial to calculate if scans overlap, but I think if both conditions are highly > > selective then we have less chance for scans to overlap, don't we?
   
   This comment is the same 2 ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283361954


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Indeed, the suggested merged version is still worse than the original individual queries when scans don't overlap as the merged version calculates both conditions for rows of both partitions while the individual original queries didn't do that.
   But can we write a merged query that doesn't have this flaw when scans don't overlap?
   
   I just realized that I've might misunderstood your comment:
   
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   Are we on the same page that the merged query of such patterns must contain a `Filter <condition 1> OR <condition 2>` node between the aggregate and the scan?
   E.g. the
   ```
   SELECT
     avg(power) avg_power FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     count(power) count_power FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     count(DISTINCT power) FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     avg(power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen")),
     count(power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen")),
     count(DISTINCT power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen"))
   FROM data
   ```
   merged example in the ticket description is wrong.
   IMO we should keep the filter between the aggregate and scan nodes in the merged query, otherwise we can't push down the combined `<condition 1> OR <condition 2>` filter to the scan and the scan might return much more data. (`FileSourceStrategy.unapply() / `ScanOperation.unapply()` can't extract filters from aggregates directly, can it?)



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1662075897

   @beliefer, I've upodated my PR with comments to ellaborate on how it handles `Filter`s: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR351-R524


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283069709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Do you mean the merged number of aggregates? How does that help to avoid https://github.com/apache/spark/pull/42223#discussion_r1282023520?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281566304


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Hm, I argued against 2., please see above.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281560378


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > make sense then I would say:
   >
   
   > - if the conditions are true literals (or they doesn't exist at all)
   
   > - or the conditions match
   
   I agree the opinion. This PR support these two cases.
   
   1. The predicate of two side are true or they doesn't exist at all.
   2. The predicate of two side are selective.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280319320


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesHelper.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+/**
+ * The helper class used to merge scalar subqueries.
+ */
+trait MergeScalarSubqueriesHelper {

Review Comment:
   @peter-toth I create this trait used to share the common functions.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280720164


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CombineJoinedAggregatesBenchmark.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.benchmark
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.internal.SQLConf.COMBINE_JOINED_AGGREGATES_ENABLED
+
+/**
+ * Benchmark to measure performance for [[CombineJoinedAggregates]] computation.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt: bin/spark-submit --class <this class>
+ *      --jars <spark core test jar>,<spark catalyst test jar> <spark sql test jar>
+ *   2. build/sbt "sql/test:runMain <this class>"
+ *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *      Results will be written to
+ *      "benchmarks/CombineJoinedAggregatesBenchmark-results.txt".
+ * }}}
+ */
+object CombineJoinedAggregatesBenchmark extends SqlBasedBenchmark {

Review Comment:
   We don't really need to add microbenchmark for optimizer features. The plan change is sufficient to show the benefits. You can keep the benchmark locally to show the perf numbers.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281532731


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   @peter-toth Thank you for the point. As you said, this situation is only worth considering when `PushDownPredicates` is disabled. AFAIK, `PushDownPredicates` is enabled and we can't disable it. This PR placed  `CombineJoinedAggregates` after `PushDownPredicates` in Optimizer list, so we could avoid to treat the scene.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1666495642

   > > But honestly, I don't like the idea that we duplicate the code of MergeScalarSubqueries.tryMergePlans() with some
   > > modifications into this new rule just because we don't want to fix agregate merging there. IMO if we all are on the same
   > > page about the discussed solution in [#42223 (comment)](https://github.com/apache/spark/pull/42223#issuecomment-1665208447) then we
   > > should update MergeScalarSubqueries.tryMergePlans() (in #37630) and use the
   > > updated common merging logic in this PR.
   > 
   > I'm also don't like the idea as you said. But it seems the code is different between this PR and #37630. Maybe we should advance these two PRs separately until a shared opportunity is discovered, and then reconstruct both.
   
   IMO the codes are very similar and the basic algorightm is identical. Both methods are just traversing the plan and comparing nodes if/how they can be merged.
   I'm off for a week, but once returned I can update https://github.com/apache/spark/pull/42223 with the discussed and maybe remove the physical scan equality check part. Although that part is useful as in some cases we don't need to use any heuristics to allow merging (https://github.com/apache/spark/pull/42223#issuecomment-1665208447), it makes my PR complex. Maybe we can add it back later...
   
   BTW this PR still doesn't use the "condition expression cost is low" heuristics that we discussed. The current "isLikelySelective" doesn't make sense (https://github.com/apache/spark/pull/42223#discussion_r1281846237) and has nothing to do with expression cost.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284113208


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Take an example, condition 1 is `flag is true` and condition 2 is `flag is false`, put these conditions into project list will add two boolean columns.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283361954


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Indeed, the suggested merged version is still worse than the original individual queries when scans don't overlap as the merged calculates both conditions for rows of both partitions while the individual original queries didn't do that.
   But I'm not sure if we can write a merged query that doesn't have this flaw when scans don't overlap...



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280716867


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map(mapAttributes(_, outputMap))
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            Seq(
+              (la.aggregateExpressions, filters.head),
+              (rightAggregateExprs, filters.last)
+            ).flatMap { case (aggregateExpressions, propagatedFilter) =>
+              aggregateExpressions.map { ne =>
+                ne.transform {
+                  case ae @ AggregateExpression(_, _, _, filterOpt, _) =>
+                    val newFilter = filterOpt.map { filter =>
+                      And(filter, propagatedFilter)
+                    }.orElse(Some(propagatedFilter))
+                    ae.copy(filter = newFilter)
+                }.asInstanceOf[NamedExpression]
+              }
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), AttributeMap.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>

Review Comment:
   do you have tests that hit this branch? Ideally `Aggregate` and `Project` will be merged.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281460766


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map(mapAttributes(_, outputMap))
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            Seq(
+              (la.aggregateExpressions, filters.head),
+              (rightAggregateExprs, filters.last)
+            ).flatMap { case (aggregateExpressions, propagatedFilter) =>
+              aggregateExpressions.map { ne =>
+                ne.transform {
+                  case ae @ AggregateExpression(_, _, _, filterOpt, _) =>
+                    val newFilter = filterOpt.map { filter =>
+                      And(filter, propagatedFilter)
+                    }.orElse(Some(propagatedFilter))
+                    ae.copy(filter = newFilter)
+                }.asInstanceOf[NamedExpression]
+              }
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), AttributeMap.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>

Review Comment:
   Yes. This test case has already been covered. Please see `test("join side is not Aggregate")`



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283307075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   That's true, merging logic in this PR is simpler (e.g. can't handle joins). But please note that currently this PR accepts different patterns as well. Infinite number of `Projects` and `Filter` nodes are accepted between the `Aggregate` and `Scan` nodes as long as both queries contains them. Although that's a bit atificial example as optimizer rules gets rid of them in most cases...
   (@beliefer, this is actually connected to https://github.com/apache/spark/pull/42223#discussion_r1279051263 that still doesn't seem correct.)
   But I'm Ok with accepting only this simple pattern like aggregates for merging.
   
   > I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   I can't agree more.
   
   > Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold.
   
   Hm, or we could also avoid double evaluation by aliasing conditions like:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```
   Actually I already do a very similar thing when I propagate up a filter condition through a `Project`: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR575-R577



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284057348


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Merging can always "inflate" data in case of aggregate function columns are different to filter condition columns as those condition columns need to go up to aggregate in the merged query while in the original queries they were pruned after filter. In this case if we alias the conditions to small boolean columns then shuffle data size decrease and less "stressful".
   
   The worst case you are referring to is when aggregate function columns are the superset of filter condition columns, but even in this case only small boolean columns are added and the gain is that we don't need to evaluate the filters twice.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1665208447

   > I don't think putting the predicates in a Project helps as the problem is from scan prunning.
   
   Sorry, my first comment about the extra project (https://github.com/apache/spark/pull/42223#discussion_r1283307075 was confusing. I got that argument, and I agreed with you (https://github.com/apache/spark/pull/42223#discussion_r1283361954).
   
   But if we decided to merge the queries (based on any heuristics) then the extra project can help to avoid evaluating the filters 2 times in the merged query and actually can decrease data to be shuffled for the aggregation: https://github.com/apache/spark/pull/42223#discussion_r1284057348. IMO the extra project beween the filter and scan won't prevent the merged condition to be pushed-down to the scan so i don't see any drawbacks of it.
   
   > Given that it's hard to estimate the overlapped scan size, the only idea I can think of is to estimate the cost of the predicate. We only do the merge if the predicate is cheap to evaluate: only contains simple comparison and the expression tree size is small.
   
   That makes sense to me and I don't have any better idea. Although I still think we can and should check it any filters are pushed down to scans in the original queries. If there is no pushed-down filters or pushed-down filters fully match then we are safe to merge as scans fully overlap. If there are non-matching pushed-down filters then we can use the suggestged expression is cheap heuristics.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1665623401

   > @peter-toth Setting aside the previous discussion, after all, the current implementation is the simplest pattern, which is also an advantage that cannot be ignored. If adding a project can bring performance improvement, we can follow up and improve.
   
   If we want to push this PR for some reason I'm not against it if the above mentioned correctness issue gets fixed (https://github.com/apache/spark/pull/42223#discussion_r1279051263).
   But honestly, I don't like the idea that we duplicate the code of `MergeScalarSubqueries.tryMergePlans()` with some modifications into this new rule just because we don't want to fix agregate merging there. IMO if we all on the same page about the discussed solution in https://github.com/apache/spark/pull/42223#issuecomment-1665208447 then we should update `MergeScalarSubqueries.tryMergePlans()` (in https://github.com/apache/spark/pull/37630) and use the updated common merging logic in this PR.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1657389924

   ping @cloud-fan @viirya cc @MaxGekk @gengliangwang 


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279076644


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {
+      case j @ Join(_, _, joinType, None, _) if isSupportedJoinType(joinType) =>
+        val aggregates = ArrayBuffer.empty[Aggregate]
+        if (collectAggregate(j, aggregates)) {
+          var finalAggregate: Option[LogicalPlan] = None
+          for ((aggregate, i) <- aggregates.tail.zipWithIndex

Review Comment:
   But I'm not sure how can you replace the `Join` to the last merged aggregate only?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates [spark]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1892901786

   We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
   If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280715232


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map(mapAttributes(_, outputMap))
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            Seq(
+              (la.aggregateExpressions, filters.head),
+              (rightAggregateExprs, filters.last)
+            ).flatMap { case (aggregateExpressions, propagatedFilter) =>
+              aggregateExpressions.map { ne =>
+                ne.transform {
+                  case ae @ AggregateExpression(_, _, _, filterOpt, _) =>
+                    val newFilter = filterOpt.map { filter =>
+                      And(filter, propagatedFilter)

Review Comment:
   the old query evaluates `Filter` first, we should keep it and here it should be `And(propagatedFilter, filter)`



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281475085


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283069709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Do you mean the merged number of aggregates? How does that help to avoid https://github.com/apache/spark/pull/42223#discussion_r1282023520?
   And please also note that the more aggregates we can merge the more performance improvement we can usually get...



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284125676


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   If a conditions is `f(c)` and `c` is not used in the aggregate function (e.g. it is `sum(a)`) then the project also removes `c` besides adding a new boolean column (we can write `sum(a) FILTER (WHERE <boolean column>)` instead of `sum(a) FILTER (WHERE f(c))`) .
   If `f(c1, c2, ..., c10)` is the condition and none of these are in the aggregate function then the project removes 10 columns and adds only 1 boolean.
   What you are reffering to is the worst case, but IMO on average, the extra projects helps.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1665295635

   @peter-toth I agree that the extra project can help if we decided to merge. However, the plan pattern becomes complicated. Without the extra project, the merged aggregate is still `Aggregate -> Filter -> Scan`. We can just define a rule for merging two aggregates, and it can incrementally merge all joined aggregates.  With the extra project, we need to define how to merge `Aggregate -> Filter -> Project -> Scan` + `Aggregate -> Filter -> Project -> Scan`, or `Aggregate -> Filter -> Project -> Scan` + `Aggregate -> Filter -> Scan`
   
   I think a few extra boolean columns won't increase the shuffle size too much. When we have a partial aggregate logical plan, we can strip the added boolean columns right after partial aggregate, but before final aggregate, as the aggregate function filter is only evaluated in partial aggregate.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281537652


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   I added test case `test("join two side are Aggregates with subquery")`



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281521725


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > Thinking more about it, I think what really matters is the predicate selectivity. No filter means an always true predicate.
   > 
   > To be safe, I think we should reuse some code of DPP that determines selective predicates, and only merge aggregates if they both have selective predicates.
   
   Is that so? I think the performance gain of merging queries comes from the fact that we don't need to scan the same data 2 times but we can calculate the 2 aggregates in 1 run.
   E.g. If we have 2 queries:
   ```
   SELECT sum(a) FROM t WHERE <condition 1>
   ```
   and
   ```
   SELECT sum(b) FROM t WHERE <condition 2>
   ```
   and we want to decide when merging the queries into
   ```
   SELECT
     sum(a) FILTER (WHERE <condition 1>),
     sum(b) FILTER (WHERE <condition 2>)
   FROM t
   WHERE <condition 1> OR <condition 2>
   ```
   make sense then I would say:
   - if the conditions are true literals (or they doesn't exist at all)
   - or the conditions match
   
   then it definitely does make sense to merge them.
   But in the first case the conditions are not selective at all and in the second case condition selectivity doesn't matter. (Actually these are the 2 cases that are already covered by `MergeScalarSubqueries.tryMergePlans()`: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala#L275-L286.)
   
   Now, if the conditions differ then it isn't trivial to calculate if scans overlap, but I think if both condisions are highly selective then we have less chance for scans to overlap, don't we?
   
   In my https://github.com/apache/spark/pull/37630 I used a different heuristics to disable merging of aggregates with different filter conditions. If the conditions contain any partitioning or bucketing columns then aggregates are not merged.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281525251


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Actually, if we disable this feature by default then do we need to define any heuristics? Can't we just let the user enable the feature if needed?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279051263


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   I'm not sure this is correct as you simply ignore the propagated filters to here, but I can imagine adjacent `Filter` nodes in a plan...
   Please check https://github.com/apache/spark/pull/37630 how it handles this case, but as I mentioned IMO the plan merge code shouldn't be duplicated to here.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283212815


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold. 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283204109


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I think a difference here is this PR only merges Aggregates with this pattern
   ```
   Aggregate
     Filter
       TableScan
   ```
   
   That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283026154


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > allowed merge depending on a new spark.sql.planMerge.ignorePushedDataFilters
   
   The treatment looks too strict too. Even if the data filter cannot be pushed down, we can still achieve many benefits. 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283307075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   That's true, merging logic in this PR is simpler (e.g. can't handle joins). But please note that currently this PR accepts different patterns as well. Infinite number of `Projects` and `Filter` nodes are accepted between the `Aggregate` and `Scan` nodes as long as both queries contains them. Although that's a bit atificial example as optimizer rules gets rid of them in most cases...
   (@beliefer, this is actually connected to https://github.com/apache/spark/pull/42223#discussion_r1279051263 that still doesn't seem correct.)
   But I'm Ok with this simple pattern limitation.
   
   > I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   I can't agree more.
   
   > Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold.
   
   Hm, or we could also avoid double evaluation by aliasing conditions like:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```
   Actually I already do a very similar thing when I propagate up a filter condition through a `Project`: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR575-R577



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284966377


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   I got you mean now. Let's fix it.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be benefitial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 2 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I:
   - alowed aggregate merge if no filters can be pushed down to scans,
   - disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters,
   - allowed merge depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config if filters can be pushed down as data filters
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1278949683


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.

Review Comment:
   Yeah. The comment is confused. I mean is not support aggregate expression with filter.
   I will update it.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1661787829

   > BTW, in my https://github.com/apache/spark/pull/37630 I used a different heuristics to disable merging of aggregates with > different filter conditions. If the conditions contain any partitioning or bucketing columns then aggregates are not merged.
   
   @peter-toth Could you tell me more? I can't found the treat for partitioning or bucketing columns.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279067309


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {
+      case j @ Join(_, _, joinType, None, _) if isSupportedJoinType(joinType) =>
+        val aggregates = ArrayBuffer.empty[Aggregate]
+        if (collectAggregate(j, aggregates)) {
+          var finalAggregate: Option[LogicalPlan] = None
+          for ((aggregate, i) <- aggregates.tail.zipWithIndex

Review Comment:
   Got it now thanks! Maybe we could try merging an aggregate with all the previous merged and non-merged ones.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280724429


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Thinking more about it, I think what really matters is the predicate selectivity. No filter means an always true predicate.
   
   To be safe, I think we should reuse some code of DPP that determines selective predicates, and only merge aggregates if they both have selective predicates.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1663601053

   > But my point is that it doesn't matter how the filter looks like (is it an OR condition or not). I enabled merging if only
   > FileSourceScanExec.dataFilters differ between the 2 scans. If FileSourceScanExec.partitionFilters or
   > FileSourceScanExec.optionalBucketSet differ then merging is disabled because partitioning and bucketing filters can be 
   > more selective in terms what files to scan...
   
   In theory, whether it is data filters or partition filters, there is a possibility of data overlap when connected filters with `or`.
   Before merge the filters (e.g. `p = 1`, `p = 2`), assume each partition have one file, so we need to read two partition files.
   After merge the two filters, we still need to read two partition files.
   I think the overhead of scan partition files is the same. the different is the filter need to calculates more. e.g. `p = 1` also need to treat the data come from `p = 2`.
   So, personally, I think the overhead of calculate is similar, no matter which filter is.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1665315777

   > @peter-toth I agree that the extra project can help if we decided to merge. However, the plan pattern becomes complicated. Without the extra project, the merged aggregate is still `Aggregate -> Filter -> Scan`. We can just define a rule for merging two aggregates, and it can incrementally merge all joined aggregates. With the extra project, we need to define how to merge `Aggregate -> Filter -> Project -> Scan` + `Aggregate -> Filter -> Project -> Scan`, or `Aggregate -> Filter -> Project -> Scan` + `Aggregate -> Filter -> Scan`.
   
   Those patterns are already covered in the current code: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala#L235-L255 and not touched in my aggregate merge PR: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR292-R318
   
   > Also note that, `WHERE cond1 OR cond2` can be better than `FROM (SELECT cond1 AS b_col1, cond2 AS b_col2) WHERE b_col1 OR b_col2 `, because `OR` has shortcut and `cond2` is not always evaluated for each input row. common subexpression elimination might be a better approach here, if we can make it cross the operator boundary (or whole-stage-codegen can only do it?)
   
   I'm nost sure `WHERE cond1 OR cond2` is better because up in the aggregate `cond2` is always evaluated for all rows.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1687950097

   Sorry, last week was a bit hectic for me. I've already put together a change to https://github.com/apache/spark/pull/37630 to make it simpler and contain the discussed, but haven't got time to update the PR. I will do it soon.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1660413251

   Instead of showing an example query, can you define the general form of joined aggregates that can be merged? 


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281461848


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CombineJoinedAggregatesBenchmark.scala:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.benchmark
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.internal.SQLConf.COMBINE_JOINED_AGGREGATES_ENABLED
+
+/**
+ * Benchmark to measure performance for [[CombineJoinedAggregates]] computation.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt: bin/spark-submit --class <this class>
+ *      --jars <spark core test jar>,<spark catalyst test jar> <spark sql test jar>
+ *   2. build/sbt "sql/test:runMain <this class>"
+ *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *      Results will be written to
+ *      "benchmarks/CombineJoinedAggregatesBenchmark-results.txt".
+ * }}}
+ */
+object CombineJoinedAggregatesBenchmark extends SqlBasedBenchmark {

Review Comment:
   OK



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1664916455

   For merging `func1(...) ... WHERE cond1` and `func2(...) ... WHERE cond2`, we got
   ```
   func1(...) FILTER cond1, func2(...) FILTER cond2 ... WHERE cond1 OR cond2
   ```
   
   Assuming there is no overlapped scan (so almost no benefit), and the table has N rows. Previously, both `cond1` and `cond2` get evaluated at most N times (the scan gets prunned). If they are partition filters, then they are evaluated 0 times. Now, they get evaluated at most 2N times. It's likely more than 2 times than before as we prune less data from the scan. The worst case is partition filters. Before, they get evaluated 0 times, now they get evaluated (numRows matching `cond1 OR cond2`) times, plus some extra evaluation in the aggregate filter. I don't think putting the predicates in a `Project` helps as the problem is from scan prunning.
   
   Given that it's hard to estimate the overlapped scan size, the only idea I can think of is to estimate the cost of the predicate. We only do the merge if the predicate is cheap to evaluate: only contains simple comparison and the expression tree size is small.
   
   We can define some patterns when the overlapped scan size must be large, e.g. one side has no filter, or the filters in two aggregates are the same. For these cases, we can always apply the merge.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be beneficial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 2 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I:
   - alowed aggregate merge if no filters can be pushed down to scans,
   - disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters,
   - allowed merge depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config if filters can be pushed down as data filters. (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did scan 1-1 partitions, so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be beneficial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 1-1 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I alowed aggregate merge if original query scans match or differ only in data filters depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR277-R290 (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283069709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Do you mean the merged number of aggregates? How does that help to avoid https://github.com/apache/spark/pull/42223#discussion_r1282023520?
   And please also note that the more aggregates we can merge the more benefit we can usually get...



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283307075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   That's true, merging logic in this PR is simpler (e.g. can't handle joins). But please note that currently this PR accepts different patterns as well. Infinite number of `Projects` and `Filter` nodes are accepted between the `Aggregate` and `Scan` nodes as long as both queries contains them. Although that's a bit atificial example as optimizer rules gets rid of them in most cases...
   (@beliefer, this is actually connected to https://github.com/apache/spark/pull/42223#discussion_r1279051263 that still doesn't seem correct.)
   But I'm Ok with accepting only this simple pattern like aggregates for merging.
   
   > I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   I can't agree more.
   
   > Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold.
   
   Hm, or we could also avoid double evaluation by aliasing conditions like:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```
   Actually I'm already do a very similar thing when I propagate up a filter condition through a `Project`: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR575-R577



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283316791


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > or we could also avoid double evaluation by aliasing conditions like
   
   Well then we need to consider the overlapped scan size as there is an extra `Project` between table scan and `Aggregate`.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283361954


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Indeed, the suggested merged is worse than the original individual queries when scans don't overlap.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283361954


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Indeed, the suggested merged version is still worse than the original individual queries when scans don't overlap as the merged version calculates both conditions for rows of both partitions while the individual original queries didn't do that.
   But can we write a merged query that doesn't have this flaw when scans don't overlap?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283307075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   That's true, merging logic in this PR is simpler (e.g. can't handle joins). But please note that currently this PR accepts different patterns as well. Infinite number of `Projects` and `Filter` nodes are accepted between the `Aggregate` and `Scan` nodes as long as both queries contains them. Although that's a bit atificial example as optimizer rules gets rid of them in most cases...
   (@beliefer, this is actually connected to https://github.com/apache/spark/pull/42223#discussion_r1279051263 that still doesn't seem correct.)
   But I'm Ok with accepting only this simple pattern like aggregates for merging.
   
   > I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   I can't agree more.
   
   > Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold.
   
   Hm, or we could also avoid double evaluation by aliasing conditions like:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283903546


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > merged example in the ticket description is wrong.
   
   I'm sorry. I wrote the example mistake. I corrected it already.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1662161173

   > ```
   >          case (CHECKING, FileSourceScanPlan(_, newScan), FileSourceScanPlan(_, cachedScan)) =>
   >            val (newScanToCompare, cachedScanToCompare) =
   >              if (conf.getConf(SQLConf.PLAN_MERGE_IGNORE_PUSHED_PUSHED_DATA_FILTERS)) {
   >                (newScan.copy(dataFilters = Seq.empty), cachedScan.copy(dataFilters = Seq.empty))
   >              } else {
   >                (newScan, cachedScan)
   >              }
   >            if (newScanToCompare.canonicalized == cachedScanToCompare.canonicalized) {
   >              // Physical plan is mergeable, but we still need to finish the logical merge to
   >              // propagate the filters
   >              tryMergePlans(newPlan, cachedPlan, DONE)
   >            } else {
   >              None
   >            }
   > ```
   > 
   > I think the above code is not needed. Generally, we concat the predicates with `OR`, the origin filters still could be pushed down to file sources.
   
   Please comment on the other PR regarding the code of that PR.
   But my point is that it doesn't matter how the filter looks like (is it an `OR` condition or not). I enabled merging if only `FileSourceScanExec.dataFilters` differ between the 2 scans. If `FileSourceScanExec.partitionFilters` or `FileSourceScanExec.optionalBucketSet` differ then merging is disabled because partitioning and bucketing filters can be more selective in terms what files to scan...


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did scan 1-1 partitions, so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be beneficial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 2 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I:
   - alowed aggregate merge if no filters can be pushed down to scans,
   - disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters,
   - allowed merge depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config if filters can be pushed down as data filters. (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1665607210

   @peter-toth Setting aside the previous discussion, after all, the current implementation is the simplest pattern, which is also an advantage that cannot be ignored. If adding a project can bring performance improvement, we can follow up and improve.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284041430


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   No, otimizer rules might not make plans any better if prerequisites aren't met, but a rule shouldn't create incorrect plan in any circumstances.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281521725


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > Thinking more about it, I think what really matters is the predicate selectivity. No filter means an always true predicate.
   > 
   > To be safe, I think we should reuse some code of DPP that determines selective predicates, and only merge aggregates if they both have selective predicates.
   
   Is that so? I think the performance gain of merging queries comes from the fact that we don't need to scan the same data 2 times but we can calculate the 2 aggregates in 1 run.
   E.g. If we have 2 queries:
   ```
   SELECT sum(a) FROM t WHERE <condition 1>
   ```
   and
   ```
   SELECT sum(b) FROM t WHERE <condition 2>
   ```
   and we want to decide when merging the queries into
   ```
   SELECT
     sum(a) FILTER (WHERE <condition 1>),
     sum(b) FILTER (WHERE <condition 2>)
   FROM t
   WHERE <condition 1> OR <condition 2>
   ```
   make sense then I would say:
   - if the conditions are true literals (or they doesn't exist at all)
   - or the conditions match
   
   then it definitely does make sense to merge them.
   But in the first case the conditions are not selective at all and in the second case condition selectivity doesn't matter. (Actually these are the 2 cases that are already covered by `MergeScalarSubqueries.tryMergePlans()`: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala#L275-L286.)
   
   Now, if the conditions differ then it isn't trivial to calculate if scans overlap, but I think if both conditions are highly selective then we have less chance for scans to overlap, don't we?
   
   BTW, in my https://github.com/apache/spark/pull/37630 I used a different heuristics to disable merging of aggregates with different filter conditions. If the conditions contain any partitioning or bucketing columns then aggregates are not merged.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1661832217

   > > BTW, in my #37630 I used a different heuristics to disable merging of aggregates with > different filter conditions. If the conditions contain any partitioning or bucketing columns then aggregates are not merged.
   > 
   > @peter-toth Could you tell me more? I can't found the treat for partitioning or bucketing columns.
   
   That heuristics made the whole PR complex. You can follow the logic of `ScanCheck` object and the case `case (CHECKING, FileSourceScanPlan(_, newScan), FileSourceScanPlan(_, cachedScan)) =>` (https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR277-R290) in `tryMergePlans()`. That case actually peeks into the physical plan to check if only pushed-down data filters differ (partitioning and bucketing filters do match).
   
   BTW, I'm not saying that it is the right heuristics to decide if we should merge aggregates with different filters, it is just the one I was able to come up with...
   Anyways, expecting highly selective predicates seems a bit counter intuitive to me. And as I mentioned I'm also fine with disabling the feature with a config by default and let the users enable it for some of queries that benefit from it.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281846237


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Yes, selective predicates => lower chance for scans to overlap => less improvement from merging.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281460766


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map(mapAttributes(_, outputMap))
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            Seq(
+              (la.aggregateExpressions, filters.head),
+              (rightAggregateExprs, filters.last)
+            ).flatMap { case (aggregateExpressions, propagatedFilter) =>
+              aggregateExpressions.map { ne =>
+                ne.transform {
+                  case ae @ AggregateExpression(_, _, _, filterOpt, _) =>
+                    val newFilter = filterOpt.map { filter =>
+                      And(filter, propagatedFilter)
+                    }.orElse(Some(propagatedFilter))
+                    ae.copy(filter = newFilter)
+                }.asInstanceOf[NamedExpression]
+              }
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), AttributeMap.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>

Review Comment:
   Yes. This test case has already been covered. Please see `test("join side is not Aggregate") {`



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283361954


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Indeed, the suggested merged version is still worse than the original individual queries when scans don't overlap as the merged version calculates both conditions for rows of both partitions while the individual original queries didn't do that.
   But can we write a merged query that doesn't have this flaw when scans don't overlap?
   
   I just realized that I've might misunderstood your comment:
   
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   Are we on the same page that the merged query of such patterns must contain a `Filter <condition 1> OR <condition 2>` node between the aggregate and the scan?
   E.g. the
   ```
   SELECT
     avg(power) avg_power FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     count(power) count_power FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     count(DISTINCT power) FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     avg(power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen")),
     count(power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen")),
     count(DISTINCT power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen"))
   FROM data
   ```
   merged example in the ticket description is wrong.
   IMO we should keep the filter between the aggregate and scan nodes in the merged query, otherwise we can't push down the combined `<condition 1> OR <condition 2>` filter to the scan and the scan might return much more data. (`FileSourceStrategy.unapply()` / `ScanOperation.unapply()` can't extract filters from aggregates directly, can it?)



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283307075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   That's true, merging logic in this PR is simpler (e.g. this PR can't handle joins, although my PR doesn't allow filter propagation through joins either). But please note that currently this PR accepts different patterns as well. Infinite number of `Projects` and `Filter` nodes are accepted between the `Aggregate` and `Scan` nodes as long as both queries contains them. Although that's a bit atificial example as optimizer rules gets rid of them in most cases...
   (@beliefer, this is actually connected to https://github.com/apache/spark/pull/42223#discussion_r1279051263 that still doesn't seem correct.)
   But I'm Ok with this simple pattern limitation.
   
   > I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   I can't agree more.
   
   > Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold.
   
   Hm, or we could also avoid double evaluation by aliasing conditions like:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```
   Actually I already do a very similar thing when I propagate up a filter condition through a `Project`: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR575-R577



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281430619


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.
+ * 2. The upstream node of these [[Aggregate]]s to be merged exists [[Join]].
+ */
+object CombineJoinedAggregates extends Rule[LogicalPlan] with MergeScalarSubqueriesHelper {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map(mapAttributes(_, outputMap))
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            Seq(
+              (la.aggregateExpressions, filters.head),
+              (rightAggregateExprs, filters.last)
+            ).flatMap { case (aggregateExpressions, propagatedFilter) =>
+              aggregateExpressions.map { ne =>
+                ne.transform {
+                  case ae @ AggregateExpression(_, _, _, filterOpt, _) =>
+                    val newFilter = filterOpt.map { filter =>
+                      And(filter, propagatedFilter)

Review Comment:
   It hears good to me.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1657990533

   Hi @beliefer,
   This is a nice optimization and awsome performance improvement to Q28, but I'm not sure the implementation is done the right way:
   - You probably know that `MergeScalarSubqueries` does a very similar plan merging in case of scalar subqueries so I feel this PR shouldn't reimplement the logic of `MergeScalarSubqueries.tryMergePlans()` but reuse the already available code and add the necessary improvements to it if needed. The plan merging code can also be extracted to a common place to share between the rules.
   - I guess probably the main reason why you reimplemented merge logic is to be able to propagate diferent filter conditions from `Filter` nodes up into `Aggregate` nodes as that is required for Q28, but currently that feature is missing from `MergeScalarSubqueries.tryMergePlans()`. IMO that improvement should be added to the existing code because `Q9` could also benefit from that. Please note that I've already tried to add that in SPARK-40193 / https://github.com/apache/spark/pull/37630. 
     Unfortunately I think this feature can be tricky as there might be cases when merging queries can introduce prformance degradation. E.g. such bad case is when we merge `SELECT sum(a) FROM t WHERE p = 1` and  `SELECT sum(b) FROM t WHERE p = 2` into `SELECT sum(a) FILTER (p = 1), sum(b) FILTER (p = 2) FROM t WHERE p = 1 OR p = 2` and `p` is a partitioning column as we need to process more, but we don't scan less data in the merged query. To avoid that I used a trick in https://github.com/apache/spark/pull/37630 to peek into the physical plan to check that only pushed data filters differ but partitioning and bucketing filters match. That trick made the implementation a bit complex, but we don't need to stick to that, we could also disable filter propagation during plan merging by default as this PR does.
     Anyway, my point is that probably the feature should be added to a common place (`MergeScalarSubqueries.tryMergePlans()`).
   
   So I would suggest narrowing the cope of SPARK-44571 / this PR to merge joined aggregate queries using the existing merge logic and SPARK-40193 to handle filter propagation during plan merging. I'also happy to revive https://github.com/apache/spark/pull/37630 if there is any intterrest.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279076644


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {
+      case j @ Join(_, _, joinType, None, _) if isSupportedJoinType(joinType) =>
+        val aggregates = ArrayBuffer.empty[Aggregate]
+        if (collectAggregate(j, aggregates)) {
+          var finalAggregate: Option[LogicalPlan] = None
+          for ((aggregate, i) <- aggregates.tail.zipWithIndex

Review Comment:
   ~But I'm not sure how can you replace the `Join` to the last merged aggregate only?~ Nevermind.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279041973


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {
+      case j @ Join(_, _, joinType, None, _) if isSupportedJoinType(joinType) =>
+        val aggregates = ArrayBuffer.empty[Aggregate]
+        if (collectAggregate(j, aggregates)) {
+          var finalAggregate: Option[LogicalPlan] = None
+          for ((aggregate, i) <- aggregates.tail.zipWithIndex

Review Comment:
   Seems like you are trying to merge aggregates in sequential order only. What if the first can't be merged with the rest, but the rest can be merged together?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280788640


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   I mean this case branch seems to behave incorrectly when you have adjacent filters in both plans.
   E.g. if you have:
   ```
   SELECT avg(a)
   FROM (
     SELECT * FROM t WHERE c1 = 1
   ) t
   WHERE c2 = 1
   ```
   and
   ```
   SELECT sum(b)
   FROM (
     SELECT * FROM t WHERE c1 = 2
   ) t
   WHERE c2 = 2
   ```
   and `PushDownPredicates` is disabled then you are in that situation. When you are merging the parent `Filter` nodes you shouldn't ignore the propagated filters from child `Filter` nodes.
   
   Actually https://github.com/apache/spark/pull/37630 handles that case and many others and that's why I think splitting this PR into 2 parts would be better:
   - This rule would be still useful if you used the existing `MergeScalarSubqueries.tryMergePlans()` to merge plans. It couldn't deal with Q28 immediately, but could deal with simpler aggregates without differing filters.
   - https://github.com/apache/spark/pull/37630 could be a good base for implementing the filter merging feature. We could simplify that PR and remove the tricky physical plan checking part and introduce a simple boolean flag to enable/disable the filter merging feature like this PR does. In this case both Q9 and Q28 could benefit from the new feature...



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279058393


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>
+          val rightCondition = rf.condition transform {
+            case attr: Attribute =>
+              outputMap.getOrElse(attr.canonicalized, attr)
+          }
+          val newCondition = Or(lf.condition, rightCondition)
+
+          (Filter(newCondition, newChild), outputMap, Seq(lf.condition, rightCondition))
+        }
+      case (ll: LeafNode, rl: LeafNode) =>
+        if (ll.canonicalized == rl.canonicalized) {
+          val outputMap = rl.output.zip(ll.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ll, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case (ls: SerializeFromObject, rs: SerializeFromObject) =>
+        if (ls.canonicalized == rs.canonicalized) {
+          val outputMap = rs.output.zip(ls.output).map { case (ra, la) =>
+            ra.canonicalized -> la
+          }.toMap
+
+          Some((ls, outputMap, Seq.empty))
+        } else {
+          None
+        }
+      case _ => None
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!conf.eliminateJoinByCombineAggregateEnabled) return plan
+
+    plan.transformDownWithPruning(_.containsAnyPattern(JOIN, AGGREGATE), ruleId) {
+      case j @ Join(_, _, joinType, None, _) if isSupportedJoinType(joinType) =>
+        val aggregates = ArrayBuffer.empty[Aggregate]
+        if (collectAggregate(j, aggregates)) {
+          var finalAggregate: Option[LogicalPlan] = None
+          for ((aggregate, i) <- aggregates.tail.zipWithIndex

Review Comment:
   This rule can merge any continuous part of to be merged aggregates in sequential order.
   If the first can't be merged with the rest, the rest can still be merged together.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1660120813

   @cloud-fan The CI failure is unrelated.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283361954


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   Indeed, the suggested merged version is still worse than the original individual queries when scans don't overlap as the merged version calculates both conditions for rows of both partitions while the individual original queries didn't do that.
   But can we write a merged query that doesn't have this flaw when scans don't overlap?
   
   I just realized that I've might misunderstood your comment:
   
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   Are we on the same page that the merged query of such patterns must contain a `Filter <condition 1> OR <condition 2>` node between the aggregate and the scan?
   E.g. the
   ```
   SELECT
     avg(power) avg_power FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     count(power) count_power FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     count(DISTINCT power) FILTER (country = "USA" AND (id BETWEEN 1 AND 3 OR city = "Berkeley" OR name = "Xiao")),
     avg(power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen")),
     count(power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen")),
     count(DISTINCT power) FILTER (country = "China" AND (id BETWEEN 4 AND 5 OR city = "Hangzhou" OR name = "Wenchen"))
   FROM data
   ```
   merged example in the ticket description is wrong.
   IMO we should keep the filter between the aggregate and scan nodes in the merged query, otherwise we can't push-down the combined `<condition 1> OR <condition 2>` filter to the scan and the scan might return much more data. (`FileSourceStrategy.unapply() / `ScanOperation.unapply()` can't extract filters from aggregates directly, can it?)



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283895467


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   Yeah, if users arbitrarily exclude the optimizer, then I believe most rules will execute improperly.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284125676


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   If a conditions is `f(c)` and `c` is not used in the aggregate function (e.g. it is `sum(a)`) then the project also removes `c` besides adding a new boolean column (we can write `sum(a) FILTER (WHERE <boolean column>)` instead of `sum(a) FILTER (WHERE f(c))`) .
   If (`c1, c2, ..., c10)` is the condition and none of these are in the aggregate function then the project removes 10 columns and adds only 1 boolean.
   What you are reffering to is the worst case, but IMO on average, the extra projects helps.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281825698


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > Now, if the conditions differ then it isn't trivial to calculate if scans overlap, but I think if both conditions are
   > highly selective then we have less chance for scans to overlap, don't we?
   
   This comment is the same 2 ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283024534


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters
   
   The treatment looks too strict. For partition filters, as long as the number of data rows is not too large, it can still benefit.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283057743


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   @peter-toth After the discussion offline between @cloud-fan and me, there is a way to limit the merged number.
   So the merged filter won't be too bulky, and the amount of data repeatedly processed by the filter won't be too large.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did scan 1-1 partitions, so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be beneficial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 1-1 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I alowed aggregate merge if original query scans match. But if scans don't match then:
   - disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters,
   - but still allowed merge depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config if scans differ in data filters only. (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1666404363

   > If we want to push this PR for some reason I'm not against it if the above mentioned correctness issue gets fixed (https://github.com/apache/spark/pull/42223#discussion_r1279051263).
   
   Fixed.
   
   > But honestly, I don't like the idea that we duplicate the code of MergeScalarSubqueries.tryMergePlans() with some
   > modifications into this new rule just because we don't want to fix agregate merging there. IMO if we all are on the same 
   > page about the discussed solution in https://github.com/apache/spark/pull/42223#issuecomment-1665208447 then we 
   > should update MergeScalarSubqueries.tryMergePlans() (in https://github.com/apache/spark/pull/37630) and use the 
   > updated common merging logic in this PR.
   
   I'm also don't like the idea as you said. But it seems the code is different between this PR and https://github.com/apache/spark/pull/37630. Maybe we should advance these two PRs separately until a shared opportunity is discovered, and then reconstruct both.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1666719285

   I'm not sure if we can reuse https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala#L235-L255 directly. If the `Project` is not generated by this optimization, it might contain other expensive expressions and we should stop merging.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates [spark]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates
URL: https://github.com/apache/spark/pull/42223


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1687915225

   ping @cloud-fan @peter-toth Could we continue to carry the work a step forward ?


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1658176942

   @peter-toth Thank you for the review.
   For your first point, we surely could reuse some function if there are some code is similar. If [SPARK-40193](https://issues.apache.org/jira/browse/SPARK-40193) / https://github.com/apache/spark/pull/37630. could be merged first, I will reuse them. Otherwise, this PR merged first, I will create follow up PR for reuse.
   
   For your second point, I'm OK to pass `Filter` nodes up into `Aggregate` nodes if the `tryMergePlans` can be reused. The reason about prformance degradation is `Filter` haven't better or higher selective. Please refer the benchmark in the description. So I added the config and the default value is false.
   
   Your suggestion is welcome. But I think what's the chance to reuse them? If https://github.com/apache/spark/pull/37630 merged first, I will reuse them directly. Otherwise, I will create follow up PR to fix.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1279162753


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   Currently, this PR support some of the most basic forms do not need to propagate filters here temporarily.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did scan 1-1 partitions, so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be beneficial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 1-1 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I:
   - alowed aggregate merge if no filters can be pushed down to scans,
   - disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters,
   - allowed merge depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config if filters can be pushed down as data filters. (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1282023520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I tend to agree with you that merging is beneficial in most of the cases.
   
   But a negative case, that I mentioned in https://github.com/apache/spark/pull/42223#issuecomment-1657990533, is when `<condition 1>` is `p = 1` and `<condition 2>` is `p = 2` if `p` is a partitioning column.
   This means that the original scans dont't overlap, so when we calculate the sum of both `a` and `b` columns, the scan in the merged query returns 2 times more data (both `a` and `b` columns from both partitions) compared to the original scans (`a` from partition `p = 1` and `b` from partition `p = 2`. And that data "flows through" the whole plan and gets filtered up in the aggregate node only, which comes with additional costs.
   Also, in this case we need to scan both partitions' data files once just like the original individual queries did so no benefit on scan side.
   
   But when those conditions can't be pushed down (or they can be pushed down as "only" data filters) then I think merging is more likely to be benefitial than in the previous partition filtering case. This is because we need to scan all the data files only once in the merged query while we had to scan all files 2 times with the individual queries.
   (Now, obviously the above statment doesn't hold in all cases as pushed down data filters can also help avoiding scanning through all data files.)
   
   So this is why in https://github.com/apache/spark/pull/37630 I:
   - alowed aggregate merge if no filters can be pushed down to scans,
   - disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters,
   - allowed merge depending on a new `spark.sql.planMerge.ignorePushedDataFilters` config if filters can be pushed down as data filters. (The config is ebabled by default as `Q9` with parquet files falls into this category and the improvement from merging is significant.)
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1663724814

   > > But my point is that it doesn't matter how the filter looks like (is it an OR condition or not). I enabled merging if only
   > > FileSourceScanExec.dataFilters differ between the 2 scans. If FileSourceScanExec.partitionFilters or
   > > FileSourceScanExec.optionalBucketSet differ then merging is disabled because partitioning and bucketing filters can be
   > > more selective in terms what files to scan...
   > 
   > In theory, whether it is data filters or partition filters, there is a possibility of data overlap when connected filters with `or`. Before merge the filters (e.g. `p = 1`, `p = 2`), assume each partition have one file, so we need to read two partition files. After merge the two filters, we still need to read two partition files. I think the overhead of scan partition files is the same. the different is the filter need to calculates more. e.g. `p = 1` also need to treat the data come from `p = 2`. So, personally, I think the overhead of calculate is similar, no matter which filter is.
   > 
   > The main reason for filter merging is the amount of overlapping data. For example, `F1` obtains 100 rows of data, and `F2` obtains 50 rows of data. If the 100 rows and 50 rows completely overlap, this is the best situation. `F1` on `Aggregate1` still processes 100 rows of data, while `F2` on `Aggregate2` processes an additional 50 rows, resulting in a total of 100 rows of data. The worst case scenario is that the two do not overlap at all. So `F1` on `Aggregate1` needs to process an additional 50 rows, a total of 150 rows; `F2` on `Aggregate2` processes an additional 100 rows, totaling 150 rows.
   
   Sorry @beliefer, I didn't explain all the reasoning behind my heuristics in https://github.com/apache/spark/pull/37630. I've updated https://github.com/apache/spark/pull/42223#discussion_r1282023520, please see the details there.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283035781


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > > disabled merge (to be on the safe side) if any filters are pushed down to scans as partitioning or bucketing filters
   > 
   > The treatment looks too strict. For partition filters, as long as the number of data rows is not too large, it can still benefit.
   
   Sure, merging can be beneficial (partitions filters overlap or scan cost is not significant), but I wanted to be on the safe side with the PR. We can also lift that restriction later with a more sophisticated logic.
   
   > > allowed merge depending on a new spark.sql.planMerge.ignorePushedDataFilters
   > 
   > The treatment looks too strict too. Even if the data filter cannot be pushed down, we can still achieve many benefits.
   
   If no filters are pushed down then merging is allowed, see 1st point.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283045489


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   I've just updated the last sentence to make it clear.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1283307075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   > I think a difference here is this PR only merges Aggregates with this pattern
   > 
   > ```
   > Aggregate
   >   Filter
   >     TableScan
   > ```
   > 
   > That said, `that data "flows through" the whole plan and gets filtered up in the aggregate node only` is not an issue as the table scan output goes directly to Aggregate and gets filtered.
   
   That's true, merging logic in this PR is simpler (e.g. can't handle joins). But please note that currently this PR accepts different patterns as well. Infinite number of `Projects` and `Filter` nodes are accepted between the `Aggregate` and `Scan` nodes as long as both queries contains them. Although that's a bit atificial example as optimizer rules gets rid of them in most cases...
   (@beliefer, this is actually connected to https://github.com/apache/spark/pull/42223#discussion_r1279051263 that still doesn't seem correct.)
   But I'm Ok with accepting only this simple pattern like aggregates for merging.
   
   > I think it's a bit hard to estimate the data size of "overlapped scans". The only safe case is when one filter is always `true`. For `p1 = 1` and `p2 = 1`, if we don't know the correlation of column `p1` and `p2`, we don't know the overlapped scan size.
   
   I can't agree more.
   
   > Another direction is we stop merging aggregates if the filter is costly enough. We can define that a filter expression is cheap if it only contains simple comparison checks and the expression tree size is smaller than a threshold.
   
   Hm, or we could also avoid double evaluation by aliasing conditions like:
   ```
   Aggregate sum(a) FILTER (WHERE condition1), sum(b) FILTER (WHERE condition2)
     Filter condition1 OR condition2
       Project <condition 1> as condition1, <condition 2> as condition2
         TableScan
   ```
   Actually I'm already doing very similar thing when I propagate up a filter condition through a `Project`: https://github.com/apache/spark/pull/37630/files#diff-3d3aa853c51c01216a7f7307219544a48e8c14fabe1817850e58739208bc406aR575-R577



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284125676


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   If a conditions is `f(c)` and `c` is not used in the aggregate function (e.g. it is `sum(a)`) then the project also removes `c` besides adding a new boolean column (we can write `sum(a) FILTER (WHERE <boolean column>)` instead of `sum(a) FILTER (WHERE f(c))`) .
   If `f(c1, c2, ..., c10)` is the condition and none of these are in the aggregate function then the project removes 10 columns and adds only 1 boolean.
   What you are reffering to is the worst case, but IMO on average, the extra project helps.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1284414503


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   In most cases, the `f(c)` are different. e.g. `f1(c)`, `f2(c)` and so on.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1281539066


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateJoinByCombineAggregate.scala:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: The [[Aggregate]]s to be merged must not exists filter.
+ */
+object EliminateJoinByCombineAggregate extends Rule[LogicalPlan] {
+
+  private def isSupportedJoinType(joinType: JoinType): Boolean =
+    Seq(Inner, Cross, LeftOuter, RightOuter, FullOuter).contains(joinType)
+
+  // Collect all the Aggregates from both side of single or nested Join.
+  private def collectAggregate(plan: LogicalPlan, aggregates: ArrayBuffer[Aggregate]): Boolean = {
+    var flag = true
+    if (plan.containsAnyPattern(JOIN, AGGREGATE)) {
+      plan match {
+        case Join(left: Aggregate, right: Aggregate, _, None, _)
+          if left.groupingExpressions.isEmpty && right.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          aggregates += right
+        case Join(left @ Join(_, _, joinType, None, _), right: Aggregate, _, None, _)
+          if isSupportedJoinType(joinType) && right.groupingExpressions.isEmpty &&
+            right.aggregateExpressions.forall(filterNotDefined) =>
+          flag = collectAggregate(left, aggregates)
+          aggregates += right
+        case Join(left: Aggregate, right @ Join(_, _, joinType, None, _), _, None, _)
+          if isSupportedJoinType(joinType) && left.groupingExpressions.isEmpty &&
+            left.aggregateExpressions.forall(filterNotDefined) =>
+          aggregates += left
+          flag = collectAggregate(right, aggregates)
+        // The side of Join is neither Aggregate nor Join.
+        case _ => flag = false
+      }
+    }
+
+    flag
+  }
+
+  // TODO Support aggregate expression with filter clause.
+  private def filterNotDefined(ne: NamedExpression): Boolean = {
+    ne match {
+      case Alias(ae: AggregateExpression, _) => ae.filter.isEmpty
+      case ae: AggregateExpression => ae.filter.isEmpty
+    }
+  }
+
+  // Merge the multiple Aggregates.
+  private def mergePlan(
+      left: LogicalPlan,
+      right: LogicalPlan): Option[(LogicalPlan, Map[Expression, Attribute], Seq[Expression])] = {
+    (left, right) match {
+      case (la: Aggregate, ra: Aggregate) =>
+        val mergedChildPlan = mergePlan(la.child, ra.child)
+        mergedChildPlan.map { case (newChild, outputMap, filters) =>
+          val rightAggregateExprs = ra.aggregateExpressions.map { ne =>
+            ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+          }
+
+          val mergedAggregateExprs = if (filters.length == 2) {
+            la.aggregateExpressions.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.head))
+              }.asInstanceOf[NamedExpression]
+            } ++ rightAggregateExprs.map { ne =>
+              ne.transform {
+                case ae @ AggregateExpression(_, _, _, None, _) =>
+                  ae.copy(filter = Some(filters.last))
+              }.asInstanceOf[NamedExpression]
+            }
+          } else {
+            la.aggregateExpressions ++ rightAggregateExprs
+          }
+
+          (Aggregate(Seq.empty, mergedAggregateExprs, newChild), Map.empty, Seq.empty)
+        }
+      case (lp: Project, rp: Project) =>
+        val mergedInfo = mergePlan(lp.child, rp.child)
+        val mergedProjectList = ArrayBuffer[NamedExpression](lp.projectList: _*)
+
+        mergedInfo.map { case (newChild, outputMap, filters) =>
+          val allFilterReferences = filters.flatMap(_.references)
+          val newOutputMap = (rp.projectList ++ allFilterReferences).map { ne =>
+            val mapped = ne.transform {
+              case attr: Attribute =>
+                outputMap.getOrElse(attr.canonicalized, attr)
+            }.asInstanceOf[NamedExpression]
+
+            val withoutAlias = mapped match {
+              case Alias(child, _) => child
+              case e => e
+            }
+
+            val outputAttr = mergedProjectList.find {
+              case Alias(child, _) => child semanticEquals withoutAlias
+              case e => e semanticEquals withoutAlias
+            }.getOrElse {
+              mergedProjectList += mapped
+              mapped
+            }.toAttribute
+            ne.toAttribute.canonicalized -> outputAttr
+          }.toMap
+
+          (Project(mergedProjectList.toSeq, newChild), newOutputMap, filters)
+        }
+      // TODO support only one side contains Filter
+      case (lf: Filter, rf: Filter) =>
+        val mergedInfo = mergePlan(lf.child, rf.child)
+        mergedInfo.map { case (newChild, outputMap, _) =>

Review Comment:
   I think the user can disable `PushDownPredicates` with `spark.sql.optimizer.excludedRules` and `spark.sql.adaptive.optimizer.excludedRules` configs. And there might be other cases when we end up in that situation, that was just and example...



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] beliefer commented on pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #42223:
URL: https://github.com/apache/spark/pull/42223#issuecomment-1662121689

   ```
            case (CHECKING, FileSourceScanPlan(_, newScan), FileSourceScanPlan(_, cachedScan)) =>
              val (newScanToCompare, cachedScanToCompare) =
                if (conf.getConf(SQLConf.PLAN_MERGE_IGNORE_PUSHED_PUSHED_DATA_FILTERS)) {
                  (newScan.copy(dataFilters = Seq.empty), cachedScan.copy(dataFilters = Seq.empty))
                } else {
                  (newScan, cachedScan)
                }
              if (newScanToCompare.canonicalized == cachedScanToCompare.canonicalized) {
                // Physical plan is mergeable, but we still need to finish the logical merge to
                // propagate the filters
                tryMergePlans(newPlan, cachedPlan, DONE)
              } else {
                None
              }
   ```
   I think the above code is not needed. Generally, we concat the predicates with `OR`, the origin filters still could be pushed down to file sources.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] peter-toth commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "peter-toth (via GitHub)" <gi...@apache.org>.
peter-toth commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1285042542


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CombineJoinedAggregates.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeMap, Expression, NamedExpression, Or}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftOuter, RightOuter}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, LogicalPlan, Project, SerializeFromObject}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, JOIN}
+
+/**
+ * This rule eliminates the [[Join]] if all the join side are [[Aggregate]]s by combine these
+ * [[Aggregate]]s. This rule also support the nested [[Join]], as long as all the join sides for
+ * every [[Join]] are [[Aggregate]]s.
+ *
+ * Note: this rule doesn't following cases:
+ * 1. One of the to be merged two [[Aggregate]]s with child [[Filter]] and the other one is not.

Review Comment:
   That's ok. It is not the conditions (`f(c)`s) relation to each other what matters, but the filter columns set (`c1` ... `c10`) relation to aggregate function column set (`a`). 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42223: [SPARK-44571][SQL] Eliminate the Join by combine multiple Aggregates

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42223:
URL: https://github.com/apache/spark/pull/42223#discussion_r1280722318


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesHelper.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+/**
+ * The helper class used to merge scalar subqueries.
+ */
+trait MergeScalarSubqueriesHelper {
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.

Review Comment:
   what does `cached version` mean? I think this method will use the left query as the main query and return the attribute map to rewrite the right query.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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