From ca81109bf8e082b9af9d5b95e8c2780e49e14c72 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Thu, 19 Dec 2024 22:51:51 +0800 Subject: [PATCH 01/18] Avoid extra expression duplication when push filter --- .../spark/sql/catalyst/expressions/With.scala | 9 ++++ .../sql/catalyst/optimizer/Optimizer.scala | 41 ++++++++++++++++--- .../optimizer/FilterPushdownSuite.scala | 32 ++++++++++++++- .../InferFiltersFromConstraintsSuite.scala | 21 +++++++--- .../spark/sql/execution/SparkOptimizer.scala | 3 ++ 5 files changed, 94 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala index 5f6f9afa5797a..70d8601b02d2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala @@ -112,6 +112,15 @@ object With { With(replaced(commonExprRefs), commonExprDefs) } + def apply(expr: Expression, replaceMap: Map[Attribute, Expression]): With = { + val commonExprDefsMap = replaceMap.map(m => m._1 -> CommonExpressionDef(m._2)) + val commonExprRefsMap = commonExprDefsMap.map(m => m._1 -> new CommonExpressionRef(m._2)) + val replaced = expr.transform { + case a: Attribute if commonExprRefsMap.contains(a) => commonExprRefsMap(a) + } + With(replaced, commonExprDefsMap.values.toSeq) + } + private[sql] def childContainsUnsupportedAggExpr(withExpr: With): Boolean = { lazy val commonExprIds = withExpr.defs.map(_.id).toSet withExpr.child.exists { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b141d2be04c32..89233abf0390c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -161,6 +161,9 @@ abstract class Optimizer(catalogManager: CatalogManager) val operatorOptimizationBatch: Seq[Batch] = Seq( Batch("Operator Optimization before Inferring Filters", fixedPoint, operatorOptimizationRuleSet: _*), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression, + CollapseProject), Batch("Infer Filters", Once, InferFiltersFromGenerate, InferFiltersFromConstraints), @@ -168,7 +171,10 @@ abstract class Optimizer(catalogManager: CatalogManager) operatorOptimizationRuleSet: _*), Batch("Push extra predicate through join", fixedPoint, PushExtraPredicateThroughJoin, - PushDownPredicates)) + PushDownPredicates), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression, + CollapseProject)) val batches: Seq[Batch] = flattenBatches(Seq( Batch("Finish Analysis", FixedPoint(1), FinishAnalysis), @@ -1808,16 +1814,17 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe // state and all the input rows processed before. In another word, the order of input rows // matters for non-deterministic expressions, while pushing down predicates changes the order. // This also applies to Aggregate. - case Filter(condition, project @ Project(fields, grandChild)) + case Filter(condition, project@Project(fields, grandChild)) if fields.forall(_.deterministic) && canPushThroughCondition(grandChild, condition) => val aliasMap = getAliasMap(project) - project.copy(child = Filter(replaceAlias(condition, aliasMap), grandChild)) + val replacedByWith = rewriteExpressionByWith(condition, aliasMap) + project.copy(child = Filter(replaceAlias(replacedByWith, aliasMap), grandChild)) // We can push down deterministic predicate through Aggregate, including throwable predicate. // If we can push down a filter through Aggregate, it means the filter only references the // grouping keys or constants. The Aggregate operator can't reduce distinct values of grouping // keys so the filter won't see any new data after push down. - case filter @ Filter(condition, aggregate: Aggregate) + case filter@Filter(condition, aggregate: Aggregate) if aggregate.aggregateExpressions.forall(_.deterministic) && aggregate.groupingExpressions.nonEmpty => val aliasMap = getAliasMap(aggregate) @@ -1831,8 +1838,8 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } if (pushDown.nonEmpty) { - val pushDownPredicate = pushDown.reduce(And) - val replaced = replaceAlias(pushDownPredicate, aliasMap) + val replacedByWith = rewriteExpressionsByWith(pushDown, aliasMap) + val replaced = replaceAlias(replacedByWith.reduce(And), aliasMap) val newAggregate = aggregate.copy(child = Filter(replaced, aggregate.child)) // If there is no more filter to stay up, just eliminate the filter. // Otherwise, create "Filter(stayUp) <- Aggregate <- Filter(pushDownPredicate)". @@ -1978,6 +1985,28 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe case _ => false } } + + private def rewriteExpressionsByWith( + exprs: Seq[Expression], + aliasMap: AttributeMap[Alias]): Seq[Expression] = { + exprs.map(rewriteExpressionByWith(_, aliasMap)) + } + + private def rewriteExpressionByWith( + expr: Expression, + aliasMap: AttributeMap[Alias]): Expression = { + val replaceMap = expr.collect { case a: Attribute => a } + .groupBy(identity) + .transform((_, v) => v.size) + .filter(_._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => !CollapseProject.isCheap(m._2)) + if (replaceMap.nonEmpty) { + With(expr, replaceMap) + } else { + expr + } + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 9424ecda0ed8b..70c57dd41371a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -45,7 +45,10 @@ class FilterPushdownSuite extends PlanTest { CollapseProject) :: Batch("Push extra predicate through join", FixedPoint(10), PushExtraPredicateThroughJoin, - PushDownPredicates) :: Nil + PushDownPredicates) :: + Batch("Rewrite With expression", FixedPoint(10), + RewriteWithExpression, + CollapseProject) :: Nil } val attrA = $"a".int @@ -1539,4 +1542,31 @@ class FilterPushdownSuite extends PlanTest { .analyze comparePlans(optimizedQueryWithoutStep, correctAnswer) } + + test("SPARK-49202: avoid extra expression duplication when push filter") { + // through project + val originalQuery1 = testRelation + .select($"a" + $"b" as "add") + .where($"add" + $"add" > 10) + val optimized1 = Optimize.execute(originalQuery1.analyze) + val correctAnswer1 = testRelation + .select($"a", $"b", $"c", $"a" + $"b" as "_common_expr_0") + .where($"_common_expr_0" + $"_common_expr_0" > 10) + .select($"a" + $"b" as "add") + .analyze + comparePlans(optimized1, correctAnswer1) + + // through aggregate + val originalQuery2 = testRelation + .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") + .where($"add" + $"add" > 10) + val optimized2 = Optimize.execute(originalQuery2.analyze) + val correctAnswer2 = testRelation + .select($"a", $"b", $"c", $"a" + $"a" as "_common_expr_1") + .where($"_common_expr_1" + $"_common_expr_1" > 10) + .select($"a", $"b", $"c") + .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") + .analyze + comparePlans(optimized2, correctAnswer2) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index d8d8a2b333bcd..e62690a7c1fd5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -30,14 +30,24 @@ class InferFiltersFromConstraintsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("InferAndPushDownFilters", FixedPoint(100), + Batch("Push Down Filters before Inferring Filters", FixedPoint(100), PushPredicateThroughJoin, - PushPredicateThroughNonJoin, + PushPredicateThroughNonJoin) :: + Batch("rewrite With expression", FixedPoint(100), + RewriteWithExpression, + CollapseProject):: + Batch("Infer Filters", Once, InferFiltersFromConstraints, CombineFilters, SimplifyBinaryComparison, BooleanSimplification, - PruneFilters) :: Nil + PruneFilters) :: + Batch("Push Down Filters", FixedPoint(100), + PushPredicateThroughJoin, + PushPredicateThroughNonJoin) :: + Batch("Rewrite With expression", FixedPoint(100), + RewriteWithExpression, + CollapseProject) :: Nil } val testRelation = LocalRelation($"a".int, $"b".int, $"c".int) @@ -151,8 +161,9 @@ class InferFiltersFromConstraintsSuite extends PlanTest { .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) .analyze val correctAnswer = t1 - .where(IsNotNull($"a") && IsNotNull(Coalesce(Seq($"a", $"b"))) && - $"a" === Coalesce(Seq($"a", $"b"))) + .select($"a", $"b", $"c", Coalesce(Seq($"a", $"b")).as("_common_expr_0")) + .where(IsNotNull($"a") && IsNotNull($"_common_expr_0") && + $"a" === $"_common_expr_0") .select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") .join(t2.where(IsNotNull($"a")), Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 6ceb363b41aef..dd41f13b0b91f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -65,6 +65,9 @@ class SparkOptimizer( RewriteDistinctAggregates), Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression, + CollapseProject), Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, // cleanup the unnecessary TrueLiteral predicates From 8b17a9f0d3e75f3718b517ddd5dc2dd0d33868c0 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Thu, 19 Dec 2024 23:18:24 +0800 Subject: [PATCH 02/18] fix --- .../sql/catalyst/optimizer/Optimizer.scala | 9 ++++---- .../InferFiltersFromConstraintsSuite.scala | 21 +++++-------------- 2 files changed, 10 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 89233abf0390c..c23bfad4cf6d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1814,17 +1814,18 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe // state and all the input rows processed before. In another word, the order of input rows // matters for non-deterministic expressions, while pushing down predicates changes the order. // This also applies to Aggregate. - case Filter(condition, project@Project(fields, grandChild)) + case Filter(condition, project @ Project(fields, grandChild)) if fields.forall(_.deterministic) && canPushThroughCondition(grandChild, condition) => val aliasMap = getAliasMap(project) - val replacedByWith = rewriteExpressionByWith(condition, aliasMap) - project.copy(child = Filter(replaceAlias(replacedByWith, aliasMap), grandChild)) + val replacedByWith = + rewriteExpressionsByWith(splitConjunctivePredicates(condition), aliasMap) + project.copy(child = Filter(replaceAlias(replacedByWith.reduce(And), aliasMap), grandChild)) // We can push down deterministic predicate through Aggregate, including throwable predicate. // If we can push down a filter through Aggregate, it means the filter only references the // grouping keys or constants. The Aggregate operator can't reduce distinct values of grouping // keys so the filter won't see any new data after push down. - case filter@Filter(condition, aggregate: Aggregate) + case filter @ Filter(condition, aggregate: Aggregate) if aggregate.aggregateExpressions.forall(_.deterministic) && aggregate.groupingExpressions.nonEmpty => val aliasMap = getAliasMap(aggregate) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index e62690a7c1fd5..d8d8a2b333bcd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -30,24 +30,14 @@ class InferFiltersFromConstraintsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Push Down Filters before Inferring Filters", FixedPoint(100), + Batch("InferAndPushDownFilters", FixedPoint(100), PushPredicateThroughJoin, - PushPredicateThroughNonJoin) :: - Batch("rewrite With expression", FixedPoint(100), - RewriteWithExpression, - CollapseProject):: - Batch("Infer Filters", Once, + PushPredicateThroughNonJoin, InferFiltersFromConstraints, CombineFilters, SimplifyBinaryComparison, BooleanSimplification, - PruneFilters) :: - Batch("Push Down Filters", FixedPoint(100), - PushPredicateThroughJoin, - PushPredicateThroughNonJoin) :: - Batch("Rewrite With expression", FixedPoint(100), - RewriteWithExpression, - CollapseProject) :: Nil + PruneFilters) :: Nil } val testRelation = LocalRelation($"a".int, $"b".int, $"c".int) @@ -161,9 +151,8 @@ class InferFiltersFromConstraintsSuite extends PlanTest { .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) .analyze val correctAnswer = t1 - .select($"a", $"b", $"c", Coalesce(Seq($"a", $"b")).as("_common_expr_0")) - .where(IsNotNull($"a") && IsNotNull($"_common_expr_0") && - $"a" === $"_common_expr_0") + .where(IsNotNull($"a") && IsNotNull(Coalesce(Seq($"a", $"b"))) && + $"a" === Coalesce(Seq($"a", $"b"))) .select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") .join(t2.where(IsNotNull($"a")), Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) From 808752e58b4d7a4f51b5fe8bb108fdbee0b173ad Mon Sep 17 00:00:00 2001 From: zml1206 Date: Fri, 20 Dec 2024 10:51:16 +0800 Subject: [PATCH 03/18] fix --- .../spark/sql/catalyst/expressions/With.scala | 6 ++- .../sql/catalyst/optimizer/Optimizer.scala | 42 +++++++++++++------ .../spark/sql/execution/SparkOptimizer.scala | 3 ++ 3 files changed, 36 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala index 70d8601b02d2f..954e0056addf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala @@ -114,9 +114,11 @@ object With { def apply(expr: Expression, replaceMap: Map[Attribute, Expression]): With = { val commonExprDefsMap = replaceMap.map(m => m._1 -> CommonExpressionDef(m._2)) - val commonExprRefsMap = commonExprDefsMap.map(m => m._1 -> new CommonExpressionRef(m._2)) + val commonExprRefsMap = + AttributeMap(commonExprDefsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) val replaced = expr.transform { - case a: Attribute if commonExprRefsMap.contains(a) => commonExprRefsMap(a) + case a: Attribute if commonExprRefsMap.contains(a) => + commonExprRefsMap.get(a).get } With(replaced, commonExprDefsMap.values.toSeq) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c23bfad4cf6d5..2980dee293666 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1817,9 +1817,8 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe case Filter(condition, project @ Project(fields, grandChild)) if fields.forall(_.deterministic) && canPushThroughCondition(grandChild, condition) => val aliasMap = getAliasMap(project) - val replacedByWith = - rewriteExpressionsByWith(splitConjunctivePredicates(condition), aliasMap) - project.copy(child = Filter(replaceAlias(replacedByWith.reduce(And), aliasMap), grandChild)) + val replacedByWith = rewriteConditionByWith(condition, aliasMap) + project.copy(child = Filter(replaceAlias(replacedByWith, aliasMap), grandChild)) // We can push down deterministic predicate through Aggregate, including throwable predicate. // If we can push down a filter through Aggregate, it means the filter only references the @@ -1839,7 +1838,7 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } if (pushDown.nonEmpty) { - val replacedByWith = rewriteExpressionsByWith(pushDown, aliasMap) + val replacedByWith = rewriteConditionByWith(pushDown, aliasMap) val replaced = replaceAlias(replacedByWith.reduce(And), aliasMap) val newAggregate = aggregate.copy(child = Filter(replaced, aggregate.child)) // If there is no more filter to stay up, just eliminate the filter. @@ -1987,26 +1986,43 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } } - private def rewriteExpressionsByWith( - exprs: Seq[Expression], + private def rewriteConditionByWith( + cond: Seq[Expression], aliasMap: AttributeMap[Alias]): Seq[Expression] = { - exprs.map(rewriteExpressionByWith(_, aliasMap)) + if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { + cond.map(rewriteByWith(_, aliasMap)) + } else cond } - private def rewriteExpressionByWith( + private def rewriteConditionByWith( + cond: Expression, + aliasMap: AttributeMap[Alias]): Expression = { + if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { + splitConjunctivePredicates(cond).map(rewriteByWith(_, aliasMap)).reduce(And) + } else cond + } + + // With does not support inline subquery + private def canRewriteByWith(expr: Expression): Boolean = { + !expr.containsPattern(PLAN_EXPRESSION) + } + + private def rewriteByWith( expr: Expression, aliasMap: AttributeMap[Alias]): Expression = { + if (!canRewriteByWith(expr)) { + return expr + } val replaceMap = expr.collect { case a: Attribute => a } .groupBy(identity) .transform((_, v) => v.size) - .filter(_._2 > 1) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) .filter(m => !CollapseProject.isCheap(m._2)) - if (replaceMap.nonEmpty) { - With(expr, replaceMap) - } else { - expr + if (replaceMap.isEmpty) { + return expr } + With(expr, replaceMap) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index dd41f13b0b91f..db7efa8088b11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -91,6 +91,9 @@ class SparkOptimizer( PushPredicateThroughNonJoin, PushProjectionThroughLimit, RemoveNoopOperators), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression, + CollapseProject), Batch("Infer window group limit", Once, InferWindowGroupLimit, LimitPushDown, From 53b18528b95428a5600c83e77cdd49d037d955ac Mon Sep 17 00:00:00 2001 From: zml1206 Date: Fri, 20 Dec 2024 14:03:07 +0800 Subject: [PATCH 04/18] fix --- .../spark/sql/catalyst/expressions/With.scala | 5 + .../sql/catalyst/optimizer/Optimizer.scala | 4 + .../optimizer/FilterPushdownSuite.scala | 53 ++++----- .../approved-plans-v1_4/q9/simplified.txt | 105 +++++++++--------- 4 files changed, 87 insertions(+), 80 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala index 954e0056addf0..b35f0272aa166 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala @@ -112,6 +112,11 @@ object With { With(replaced(commonExprRefs), commonExprDefs) } + /** + * Helper function to create a [[With]] statement when push down filter. + * @param expr original expression + * @param replaceMap Replaced attributes and common expressions + */ def apply(expr: Expression, replaceMap: Map[Attribute, Expression]): With = { val commonExprDefsMap = replaceMap.map(m => m._1 -> CommonExpressionDef(m._2)) val commonExprRefsMap = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 2980dee293666..b86b2cef3d446 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2007,6 +2007,10 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe !expr.containsPattern(PLAN_EXPRESSION) } + /** + * Use [[With]] to rewrite expression which contains attribute that are not cheap and be consumed + * multiple times. + */ private def rewriteByWith( expr: Expression, aliasMap: AttributeMap[Alias]): Expression = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 70c57dd41371a..71ede59559384 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -1543,30 +1544,32 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimizedQueryWithoutStep, correctAnswer) } - test("SPARK-49202: avoid extra expression duplication when push filter") { - // through project - val originalQuery1 = testRelation - .select($"a" + $"b" as "add") - .where($"add" + $"add" > 10) - val optimized1 = Optimize.execute(originalQuery1.analyze) - val correctAnswer1 = testRelation - .select($"a", $"b", $"c", $"a" + $"b" as "_common_expr_0") - .where($"_common_expr_0" + $"_common_expr_0" > 10) - .select($"a" + $"b" as "add") - .analyze - comparePlans(optimized1, correctAnswer1) - - // through aggregate - val originalQuery2 = testRelation - .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") - .where($"add" + $"add" > 10) - val optimized2 = Optimize.execute(originalQuery2.analyze) - val correctAnswer2 = testRelation - .select($"a", $"b", $"c", $"a" + $"a" as "_common_expr_1") - .where($"_common_expr_1" + $"_common_expr_1" > 10) - .select($"a", $"b", $"c") - .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") - .analyze - comparePlans(optimized2, correctAnswer2) + test("SPARK-50589: avoid extra expression duplication when push filter") { + withSQLConf(SQLConf.USE_COMMON_EXPR_ID_FOR_ALIAS.key -> "false") { + // through project + val originalQuery1 = testRelation + .select($"a" + $"b" as "add") + .where($"add" + $"add" > 10) + val optimized1 = Optimize.execute(originalQuery1.analyze) + val correctAnswer1 = testRelation + .select($"a", $"b", $"c", $"a" + $"b" as "_common_expr_0") + .where($"_common_expr_0" + $"_common_expr_0" > 10) + .select($"a" + $"b" as "add") + .analyze + comparePlans(optimized1, correctAnswer1) + + // through aggregate + val originalQuery2 = testRelation + .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") + .where($"add" + $"add" > 10) + val optimized2 = Optimize.execute(originalQuery2.analyze) + val correctAnswer2 = testRelation + .select($"a", $"b", $"c", $"a" + $"a" as "_common_expr_0") + .where($"_common_expr_0" + $"_common_expr_0" > 10) + .select($"a", $"b", $"c") + .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") + .analyze + comparePlans(optimized2, correctAnswer2) + } } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 817fb0007f343..269aadd62e5b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -2,77 +2,72 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 Filter [r_reason_sk] From ea68366741171da24c1596a91aa32c4f997789ba Mon Sep 17 00:00:00 2001 From: zml1206 Date: Fri, 20 Dec 2024 16:08:58 +0800 Subject: [PATCH 05/18] fix --- .../spark/sql/execution/SparkOptimizer.scala | 3 +- .../approved-plans-v1_4/q9/simplified.txt | 105 +++++++++--------- 2 files changed, 56 insertions(+), 52 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index db7efa8088b11..1f2bed4dff363 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -66,8 +66,7 @@ class SparkOptimizer( Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates), Batch("Rewrite With expression", fixedPoint, - RewriteWithExpression, - CollapseProject), + RewriteWithExpression), Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, // cleanup the unnecessary TrueLiteral predicates diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 269aadd62e5b1..817fb0007f343 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -2,72 +2,77 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 Filter [r_reason_sk] From b212228f93ecbec62fd91eddad9d32d6a3564706 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Fri, 20 Dec 2024 16:23:09 +0800 Subject: [PATCH 06/18] fix --- .../scala/org/apache/spark/sql/execution/SparkOptimizer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 1f2bed4dff363..86a59d2e5e841 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -91,8 +91,7 @@ class SparkOptimizer( PushProjectionThroughLimit, RemoveNoopOperators), Batch("Rewrite With expression", fixedPoint, - RewriteWithExpression, - CollapseProject), + RewriteWithExpression), Batch("Infer window group limit", Once, InferWindowGroupLimit, LimitPushDown, From b9df556b5cea2d4657aed21bae92a6c76a8cb1e4 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Sat, 21 Dec 2024 12:25:03 +0800 Subject: [PATCH 07/18] update --- .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 6 ++---- .../org/apache/spark/sql/execution/SparkOptimizer.scala | 8 +++----- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b86b2cef3d446..7ca920dbe8739 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -161,9 +161,6 @@ abstract class Optimizer(catalogManager: CatalogManager) val operatorOptimizationBatch: Seq[Batch] = Seq( Batch("Operator Optimization before Inferring Filters", fixedPoint, operatorOptimizationRuleSet: _*), - Batch("Rewrite With expression", fixedPoint, - RewriteWithExpression, - CollapseProject), Batch("Infer Filters", Once, InferFiltersFromGenerate, InferFiltersFromConstraints), @@ -275,7 +272,8 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveNoopOperators), // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers), - Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression))) + Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression), + )) // remove any batches with no rules. this may happen when subclasses do not add optional rules. batches.filter(_.rules.nonEmpty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 86a59d2e5e841..cb420b59a1aab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -65,8 +65,6 @@ class SparkOptimizer( RewriteDistinctAggregates), Batch("Pushdown Filters from PartitionPruning", fixedPoint, PushDownPredicates), - Batch("Rewrite With expression", fixedPoint, - RewriteWithExpression), Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, // cleanup the unnecessary TrueLiteral predicates @@ -90,8 +88,6 @@ class SparkOptimizer( PushPredicateThroughNonJoin, PushProjectionThroughLimit, RemoveNoopOperators), - Batch("Rewrite With expression", fixedPoint, - RewriteWithExpression), Batch("Infer window group limit", Once, InferWindowGroupLimit, LimitPushDown, @@ -99,7 +95,9 @@ class SparkOptimizer( EliminateLimits, ConstantFolding), Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*), - Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition))) + Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression))) override def nonExcludableRules: Seq[String] = super.nonExcludableRules ++ Seq( From 5acc5fad592b6b5ffeb328bbd0668e7f325f4184 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Sat, 21 Dec 2024 12:25:49 +0800 Subject: [PATCH 08/18] update --- .../scala/org/apache/spark/sql/execution/SparkOptimizer.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index cb420b59a1aab..65a8bf85c288a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -97,7 +97,8 @@ class SparkOptimizer( Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*), Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition), Batch("Rewrite With expression", fixedPoint, - RewriteWithExpression))) + RewriteWithExpression, + CollapseProject))) override def nonExcludableRules: Seq[String] = super.nonExcludableRules ++ Seq( From 044ffaa9e2745c6a18b46831ae93a09f8bda3647 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Sat, 21 Dec 2024 17:50:49 +0800 Subject: [PATCH 09/18] fix --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 7ca920dbe8739..3775ff60f425d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -272,8 +272,7 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveNoopOperators), // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers), - Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression), - )) + Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression))) // remove any batches with no rules. this may happen when subclasses do not add optional rules. batches.filter(_.rules.nonEmpty) From f1aa618d18e715a4d9279f82e17ee2d039ab39f0 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Mon, 23 Dec 2024 16:43:43 +0800 Subject: [PATCH 10/18] optimize split condition --- .../spark/sql/catalyst/expressions/predicates.scala | 8 ++++++++ .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 4 ++-- .../sql/catalyst/optimizer/FilterPushdownSuite.scala | 8 ++++---- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 5e402fa2b6caa..6b3849ae0c875 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -107,6 +107,14 @@ trait PredicateHelper extends AliasHelper with Logging { } } + /** + * First split the predicates by And, then combine the predicates with the same references. + */ + protected def splitAndCombinePredicates(condition: Expression): Seq[Expression] = { + val split = splitConjunctivePredicates(condition) + split.groupBy(_.references).map(_._2.reduce(And)).toSeq + } + /** * Find the origin of where the input references of expression exp were scanned in the tree of * plan, and if they originate from a single leaf node. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3775ff60f425d..e5e2e27d74e6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1828,7 +1828,7 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe // For each filter, expand the alias and check if the filter can be evaluated using // attributes produced by the aggregate operator's child operator. - val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => + val (pushDown, stayUp) = splitAndCombinePredicates(condition).partition { cond => val replaced = replaceAlias(cond, aliasMap) cond.deterministic && cond.references.nonEmpty && replaced.references.subsetOf(aggregate.child.outputSet) @@ -1995,7 +1995,7 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe cond: Expression, aliasMap: AttributeMap[Alias]): Expression = { if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - splitConjunctivePredicates(cond).map(rewriteByWith(_, aliasMap)).reduce(And) + splitAndCombinePredicates(cond).map(rewriteByWith(_, aliasMap)).reduce(And) } else cond } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 71ede59559384..dfca0c0f0e11d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -1549,11 +1549,11 @@ class FilterPushdownSuite extends PlanTest { // through project val originalQuery1 = testRelation .select($"a" + $"b" as "add") - .where($"add" + $"add" > 10) + .where($"add" + $"add" > 10 && $"add" < 10) val optimized1 = Optimize.execute(originalQuery1.analyze) val correctAnswer1 = testRelation .select($"a", $"b", $"c", $"a" + $"b" as "_common_expr_0") - .where($"_common_expr_0" + $"_common_expr_0" > 10) + .where($"_common_expr_0" + $"_common_expr_0" > 10 && $"_common_expr_0" < 10) .select($"a" + $"b" as "add") .analyze comparePlans(optimized1, correctAnswer1) @@ -1561,11 +1561,11 @@ class FilterPushdownSuite extends PlanTest { // through aggregate val originalQuery2 = testRelation .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") - .where($"add" + $"add" > 10) + .where($"add" + $"add" > 10 && $"add" < 10) val optimized2 = Optimize.execute(originalQuery2.analyze) val correctAnswer2 = testRelation .select($"a", $"b", $"c", $"a" + $"a" as "_common_expr_0") - .where($"_common_expr_0" + $"_common_expr_0" > 10) + .where($"_common_expr_0" + $"_common_expr_0" > 10 && $"_common_expr_0" < 10) .select($"a", $"b", $"c") .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") .analyze From 6247c34b5a807ff60c82a0fe9ebee60e416b6ea3 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Tue, 24 Dec 2024 13:15:21 +0800 Subject: [PATCH 11/18] optimize split condition --- .../sql/catalyst/expressions/predicates.scala | 8 --- .../optimizer/MergeWithExpression.scala | 69 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 47 ++++++------- .../optimizer/RewriteWithExpression.scala | 5 +- .../optimizer/FilterPushdownSuite.scala | 21 +++--- .../spark/sql/execution/SparkOptimizer.scala | 1 + 6 files changed, 109 insertions(+), 42 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 6b3849ae0c875..5e402fa2b6caa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -107,14 +107,6 @@ trait PredicateHelper extends AliasHelper with Logging { } } - /** - * First split the predicates by And, then combine the predicates with the same references. - */ - protected def splitAndCombinePredicates(condition: Expression): Seq[Expression] = { - val split = splitConjunctivePredicates(condition) - split.groupBy(_.references).map(_._2.reduce(And)).toSeq - } - /** * Find the origin of where the input references of expression exp were scanned in the tree of * plan, and if they originate from a single leaf node. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala new file mode 100644 index 0000000000000..61a02df72c0f2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala @@ -0,0 +1,69 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{AND, FILTER, WITH_EXPRESSION} + +/** + * Before rewrite with expression, merge with expression which has same common expression for + * avoid extra expression duplication. + */ +object MergeWithExpression extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.transformUpWithSubqueriesAndPruning(_.containsPattern(FILTER)) { + case f @ Filter(cond, _) => + val newCond = cond.transformUpWithPruning(_.containsAllPatterns(AND, WITH_EXPRESSION)) { + case And(left @ With(_, _), right @ With(_, _)) => + mergeWith(left, right) + case And(left @ With(_, _), right) => + With(And(left.child, right), left.defs) + case And(left, right @ With(_, _)) => + With(And(left, right.child), right.defs) + } + f.copy(condition = newCond) + } + } + + private def mergeWith(left: With, right: With): Expression = { + val newDefs = left.defs.toBuffer + val replaceMap = mutable.HashMap.empty[CommonExpressionId, CommonExpressionRef] + right.defs.foreach {rDef => + val index = left.defs.indexWhere(lDef => rDef.child.fastEquals(lDef.child)) + if (index == -1) { + newDefs.append(rDef) + } else { + replaceMap.put(rDef.id, new CommonExpressionRef(left.defs(index))) + } + } + val newChild = if (replaceMap.nonEmpty) { + val newRightChild = right.child.transform { + case r: CommonExpressionRef if replaceMap.contains(r.id) => + replaceMap(r.id) + } + And(left.child, newRightChild) + } else { + And(left.child, right.child) + } + With(newChild, newDefs.toSeq) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e5e2e27d74e6e..d3a4b91c8fb6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -169,6 +169,7 @@ abstract class Optimizer(catalogManager: CatalogManager) Batch("Push extra predicate through join", fixedPoint, PushExtraPredicateThroughJoin, PushDownPredicates), + Batch("Merge With expression", Once, MergeWithExpression), Batch("Rewrite With expression", fixedPoint, RewriteWithExpression, CollapseProject)) @@ -1828,15 +1829,15 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe // For each filter, expand the alias and check if the filter can be evaluated using // attributes produced by the aggregate operator's child operator. - val (pushDown, stayUp) = splitAndCombinePredicates(condition).partition { cond => + val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => val replaced = replaceAlias(cond, aliasMap) cond.deterministic && cond.references.nonEmpty && replaced.references.subsetOf(aggregate.child.outputSet) } if (pushDown.nonEmpty) { - val replacedByWith = rewriteConditionByWith(pushDown, aliasMap) - val replaced = replaceAlias(replacedByWith.reduce(And), aliasMap) + val replacedByWith = rewriteConditionByWith(pushDown.reduce(And), aliasMap) + val replaced = replaceAlias(replacedByWith, aliasMap) val newAggregate = aggregate.copy(child = Filter(replaced, aggregate.child)) // If there is no more filter to stay up, just eliminate the filter. // Otherwise, create "Filter(stayUp) <- Aggregate <- Filter(pushDownPredicate)". @@ -1983,19 +1984,23 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } } - private def rewriteConditionByWith( - cond: Seq[Expression], - aliasMap: AttributeMap[Alias]): Seq[Expression] = { - if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - cond.map(rewriteByWith(_, aliasMap)) - } else cond - } - + /** + * Use [[With]] to rewrite condition which contains attribute that are not cheap and be consumed + * multiple times. + */ private def rewriteConditionByWith( cond: Expression, aliasMap: AttributeMap[Alias]): Expression = { + val newAliasMap = cond.collect { case a: Attribute => a } + .groupBy(identity) + .transform((_, v) => v.size) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => !CollapseProject.isCheap(m._2)) if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - splitAndCombinePredicates(cond).map(rewriteByWith(_, aliasMap)).reduce(And) + splitConjunctivePredicates(cond) + .map(rewriteByWith(_, AttributeMap(newAliasMap))) + .reduce(And) } else cond } @@ -2004,26 +2009,18 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe !expr.containsPattern(PLAN_EXPRESSION) } - /** - * Use [[With]] to rewrite expression which contains attribute that are not cheap and be consumed - * multiple times. - */ private def rewriteByWith( expr: Expression, - aliasMap: AttributeMap[Alias]): Expression = { + replaceMap: Map[Attribute, Expression]): Expression = { if (!canRewriteByWith(expr)) { return expr } - val replaceMap = expr.collect { case a: Attribute => a } - .groupBy(identity) - .transform((_, v) => v.size) - .filter(m => aliasMap.contains(m._1) && m._2 > 1) - .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) - .filter(m => !CollapseProject.isCheap(m._2)) - if (replaceMap.isEmpty) { + val exprAttrSet = expr.collect { case a: Attribute => a }.toSet + val newReplaceMap = replaceMap.filter(x => exprAttrSet.contains(x._1)) + if (newReplaceMap.isEmpty) { return expr } - With(expr, replaceMap) + With(expr, newReplaceMap) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala index d0c5d8158644b..6d53f3637dccb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala @@ -96,6 +96,9 @@ object RewriteWithExpression extends Rule[LogicalPlan] { val defs = w.defs.map(rewriteWithExprAndInputPlans(_, inputPlans, isNestedWith = true)) val refToExpr = mutable.HashMap.empty[CommonExpressionId, Expression] val childProjections = Array.fill(inputPlans.length)(mutable.ArrayBuffer.empty[Alias]) + val refsCount = child.collect { case r: CommonExpressionRef => r} + .groupBy(_.id) + .transform((_, v) => v.size) defs.zipWithIndex.foreach { case (CommonExpressionDef(child, id), index) => if (id.canonicalized) { @@ -103,7 +106,7 @@ object RewriteWithExpression extends Rule[LogicalPlan] { "Cannot rewrite canonicalized Common expression definitions") } - if (CollapseProject.isCheap(child)) { + if (CollapseProject.isCheap(child) || refsCount.getOrElse(id, 0) < 2) { refToExpr(id) = child } else { val childProjectionIndex = inputPlans.indexWhere( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index dfca0c0f0e11d..b7272c57c2932 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -47,6 +47,7 @@ class FilterPushdownSuite extends PlanTest { Batch("Push extra predicate through join", FixedPoint(10), PushExtraPredicateThroughJoin, PushDownPredicates) :: + Batch("Merge With expression", Once, MergeWithExpression) :: Batch("Rewrite With expression", FixedPoint(10), RewriteWithExpression, CollapseProject) :: Nil @@ -1548,26 +1549,30 @@ class FilterPushdownSuite extends PlanTest { withSQLConf(SQLConf.USE_COMMON_EXPR_ID_FOR_ALIAS.key -> "false") { // through project val originalQuery1 = testRelation - .select($"a" + $"b" as "add") - .where($"add" + $"add" > 10 && $"add" < 10) + .select($"a" + $"b" as "add", $"a" - $"b" as "sub") + .where($"add" < 10 && $"add" + $"add" > 10 && $"sub" > 0) val optimized1 = Optimize.execute(originalQuery1.analyze) val correctAnswer1 = testRelation .select($"a", $"b", $"c", $"a" + $"b" as "_common_expr_0") - .where($"_common_expr_0" + $"_common_expr_0" > 10 && $"_common_expr_0" < 10) - .select($"a" + $"b" as "add") + .where($"_common_expr_0" < 10 && + $"_common_expr_0" + $"_common_expr_0" > 10 && + $"a" - $"b" > 0) + .select($"a" + $"b" as "add", $"a" - $"b" as "sub") .analyze comparePlans(optimized1, correctAnswer1) // through aggregate val originalQuery2 = testRelation - .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") - .where($"add" + $"add" > 10 && $"add" < 10) + .groupBy($"a")($"a", $"a" + $"a" as "add", abs($"a") as "abs", count(1) as "ct") + .where($"add" < 10 && $"add" + $"add" > 10 && $"abs" > 5) val optimized2 = Optimize.execute(originalQuery2.analyze) val correctAnswer2 = testRelation .select($"a", $"b", $"c", $"a" + $"a" as "_common_expr_0") - .where($"_common_expr_0" + $"_common_expr_0" > 10 && $"_common_expr_0" < 10) + .where($"_common_expr_0" < 10 && + $"_common_expr_0" + $"_common_expr_0" > 10 && + abs($"a") > 5) .select($"a", $"b", $"c") - .groupBy($"a")($"a", $"a" + $"a" as "add", count(1) as "ct") + .groupBy($"a")($"a", $"a" + $"a" as "add", abs($"a") as "abs", count(1) as "ct") .analyze comparePlans(optimized2, correctAnswer2) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 65a8bf85c288a..bbfc5b5194a64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -96,6 +96,7 @@ class SparkOptimizer( ConstantFolding), Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*), Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition), + Batch("Merge With expression", Once, MergeWithExpression), Batch("Rewrite With expression", fixedPoint, RewriteWithExpression, CollapseProject))) From 8fb567e6dd2ab7b073731c4509608176c2748932 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Tue, 24 Dec 2024 13:24:13 +0800 Subject: [PATCH 12/18] update --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d3a4b91c8fb6c..4c61b79b5d549 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1991,15 +1991,15 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe private def rewriteConditionByWith( cond: Expression, aliasMap: AttributeMap[Alias]): Expression = { - val newAliasMap = cond.collect { case a: Attribute => a } - .groupBy(identity) - .transform((_, v) => v.size) - .filter(m => aliasMap.contains(m._1) && m._2 > 1) - .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) - .filter(m => !CollapseProject.isCheap(m._2)) if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { + val replaceWithMap = cond.collect { case a: Attribute => a } + .groupBy(identity) + .transform((_, v) => v.size) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => !CollapseProject.isCheap(m._2)) splitConjunctivePredicates(cond) - .map(rewriteByWith(_, AttributeMap(newAliasMap))) + .map(rewriteByWith(_, AttributeMap(replaceWithMap))) .reduce(And) } else cond } @@ -2011,7 +2011,7 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe private def rewriteByWith( expr: Expression, - replaceMap: Map[Attribute, Expression]): Expression = { + replaceMap: AttributeMap[Expression]): Expression = { if (!canRewriteByWith(expr)) { return expr } From 58459d0d26ca8f21f045f5bb0ca1813e6d483e34 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Tue, 24 Dec 2024 15:08:34 +0800 Subject: [PATCH 13/18] fix --- .../sql/catalyst/optimizer/Optimizer.scala | 7 +++++- .../optimizer/FilterPushdownSuite.scala | 2 +- .../InferFiltersFromConstraintsSuite.scala | 23 +++++++++++++++---- .../RewriteWithExpressionSuite.scala | 5 ++-- .../spark/sql/execution/SparkOptimizer.scala | 2 +- 5 files changed, 29 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 4c61b79b5d549..8d532db8a0e7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -161,6 +161,11 @@ abstract class Optimizer(catalogManager: CatalogManager) val operatorOptimizationBatch: Seq[Batch] = Seq( Batch("Operator Optimization before Inferring Filters", fixedPoint, operatorOptimizationRuleSet: _*), + // With expression will destroy infer filters, so need rewrite it before infer filters. + Batch("Merge With expression", fixedPoint, MergeWithExpression), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression, + CollapseProject), Batch("Infer Filters", Once, InferFiltersFromGenerate, InferFiltersFromConstraints), @@ -169,7 +174,7 @@ abstract class Optimizer(catalogManager: CatalogManager) Batch("Push extra predicate through join", fixedPoint, PushExtraPredicateThroughJoin, PushDownPredicates), - Batch("Merge With expression", Once, MergeWithExpression), + Batch("Merge With expression", fixedPoint, MergeWithExpression), Batch("Rewrite With expression", fixedPoint, RewriteWithExpression, CollapseProject)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index b7272c57c2932..39d51cac5308e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -47,7 +47,7 @@ class FilterPushdownSuite extends PlanTest { Batch("Push extra predicate through join", FixedPoint(10), PushExtraPredicateThroughJoin, PushDownPredicates) :: - Batch("Merge With expression", Once, MergeWithExpression) :: + Batch("Merge With expression", FixedPoint(10), MergeWithExpression) :: Batch("Rewrite With expression", FixedPoint(10), RewriteWithExpression, CollapseProject) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index d8d8a2b333bcd..7b876eff1d950 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -30,14 +30,26 @@ class InferFiltersFromConstraintsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("InferAndPushDownFilters", FixedPoint(100), + Batch("PushDownFilters", FixedPoint(100), PushPredicateThroughJoin, - PushPredicateThroughNonJoin, + PushPredicateThroughNonJoin) :: + Batch("Merge With expression", FixedPoint(10), MergeWithExpression) :: + Batch("Rewrite With expression", FixedPoint(10), + RewriteWithExpression, + CollapseProject) :: + Batch("InferFilters", FixedPoint(100), InferFiltersFromConstraints, CombineFilters, SimplifyBinaryComparison, BooleanSimplification, - PruneFilters) :: Nil + PruneFilters) :: + Batch("PushDownFilters", FixedPoint(100), + PushPredicateThroughJoin, + PushPredicateThroughNonJoin) :: + Batch("Merge With expression", FixedPoint(10), MergeWithExpression) :: + Batch("Rewrite With expression", FixedPoint(10), + RewriteWithExpression, + CollapseProject) :: Nil } val testRelation = LocalRelation($"a".int, $"b".int, $"c".int) @@ -151,8 +163,9 @@ class InferFiltersFromConstraintsSuite extends PlanTest { .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) .analyze val correctAnswer = t1 - .where(IsNotNull($"a") && IsNotNull(Coalesce(Seq($"a", $"b"))) && - $"a" === Coalesce(Seq($"a", $"b"))) + .select($"a", $"b", $"c", Coalesce(Seq($"a", $"b")) as "_common_expr_0") + .where(IsNotNull($"a") && IsNotNull($"_common_expr_0") && + $"a" === $"_common_expr_0") .select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") .join(t2.where(IsNotNull($"a")), Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala index 0be6ae6494643..a3a1f9f3247c7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala @@ -140,7 +140,7 @@ class RewriteWithExpressionSuite extends PlanTest { val commonExprDef2 = CommonExpressionDef(a + a, CommonExpressionId(2)) val ref2 = new CommonExpressionRef(commonExprDef2) // The inner main expression references the outer expression - val innerExpr2 = With(ref2 + outerRef, Seq(commonExprDef2)) + val innerExpr2 = With(ref2 + ref2 + outerRef, Seq(commonExprDef2)) val outerExpr2 = With(outerRef + innerExpr2, Seq(outerCommonExprDef)) comparePlans( Optimizer.execute(testRelation.select(outerExpr2.as("col"))), @@ -152,7 +152,8 @@ class RewriteWithExpressionSuite extends PlanTest { .select(star(), (a + a).as("_common_expr_2")) // The final Project contains the final result expression, which references both common // expressions. - .select(($"_common_expr_0" + ($"_common_expr_2" + $"_common_expr_0")).as("col")) + .select(($"_common_expr_0" + + ($"_common_expr_2" + $"_common_expr_2" + $"_common_expr_0")).as("col")) .analyze ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index bbfc5b5194a64..73f8f4be3a239 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -96,7 +96,7 @@ class SparkOptimizer( ConstantFolding), Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*), Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition), - Batch("Merge With expression", Once, MergeWithExpression), + Batch("Merge With expression", fixedPoint, MergeWithExpression), Batch("Rewrite With expression", fixedPoint, RewriteWithExpression, CollapseProject))) From 776122d0a80326962af5bbdbce78f100860d7921 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Tue, 24 Dec 2024 18:18:22 +0800 Subject: [PATCH 14/18] fix fix fix fix fix --- .../spark/sql/catalyst/expressions/With.scala | 16 ---- .../optimizer/MergeWithExpression.scala | 28 +------ .../sql/catalyst/optimizer/Optimizer.scala | 53 ++++++++---- .../optimizer/RewriteWithExpression.scala | 7 +- .../InferFiltersFromConstraintsSuite.scala | 34 ++++---- .../spark/sql/execution/SparkOptimizer.scala | 5 ++ .../approved-plans-v1_4/q21.sf100/explain.txt | 82 +++++++++++-------- .../q21.sf100/simplified.txt | 76 ++++++++--------- .../approved-plans-v1_4/q21/explain.txt | 82 +++++++++++-------- .../approved-plans-v1_4/q21/simplified.txt | 76 ++++++++--------- .../scala/org/apache/spark/sql/TPCBase.scala | 2 + 11 files changed, 238 insertions(+), 223 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala index b35f0272aa166..5f6f9afa5797a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/With.scala @@ -112,22 +112,6 @@ object With { With(replaced(commonExprRefs), commonExprDefs) } - /** - * Helper function to create a [[With]] statement when push down filter. - * @param expr original expression - * @param replaceMap Replaced attributes and common expressions - */ - def apply(expr: Expression, replaceMap: Map[Attribute, Expression]): With = { - val commonExprDefsMap = replaceMap.map(m => m._1 -> CommonExpressionDef(m._2)) - val commonExprRefsMap = - AttributeMap(commonExprDefsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) - val replaced = expr.transform { - case a: Attribute if commonExprRefsMap.contains(a) => - commonExprRefsMap.get(a).get - } - With(replaced, commonExprDefsMap.values.toSeq) - } - private[sql] def childContainsUnsupportedAggExpr(withExpr: With): Boolean = { lazy val commonExprIds = withExpr.defs.map(_.id).toSet withExpr.child.exists { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala index 61a02df72c0f2..5ba9f0a7f7c62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeWithExpression.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -import scala.collection.mutable - import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule @@ -34,7 +32,8 @@ object MergeWithExpression extends Rule[LogicalPlan] { case f @ Filter(cond, _) => val newCond = cond.transformUpWithPruning(_.containsAllPatterns(AND, WITH_EXPRESSION)) { case And(left @ With(_, _), right @ With(_, _)) => - mergeWith(left, right) + val defs = (left.defs ++ right.defs).distinct + With(And(left.child, right.child), defs) case And(left @ With(_, _), right) => With(And(left.child, right), left.defs) case And(left, right @ With(_, _)) => @@ -43,27 +42,4 @@ object MergeWithExpression extends Rule[LogicalPlan] { f.copy(condition = newCond) } } - - private def mergeWith(left: With, right: With): Expression = { - val newDefs = left.defs.toBuffer - val replaceMap = mutable.HashMap.empty[CommonExpressionId, CommonExpressionRef] - right.defs.foreach {rDef => - val index = left.defs.indexWhere(lDef => rDef.child.fastEquals(lDef.child)) - if (index == -1) { - newDefs.append(rDef) - } else { - replaceMap.put(rDef.id, new CommonExpressionRef(left.defs(index))) - } - } - val newChild = if (replaceMap.nonEmpty) { - val newRightChild = right.child.transform { - case r: CommonExpressionRef if replaceMap.contains(r.id) => - replaceMap(r.id) - } - And(left.child, newRightChild) - } else { - And(left.child, right.child) - } - With(newChild, newDefs.toSeq) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 8d532db8a0e7f..e0143a5bf225d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1841,8 +1841,8 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } if (pushDown.nonEmpty) { - val replacedByWith = rewriteConditionByWith(pushDown.reduce(And), aliasMap) - val replaced = replaceAlias(replacedByWith, aliasMap) + val replacedByWith = rewriteConditionByWith(pushDown, aliasMap) + val replaced = replaceAlias(replacedByWith.reduce(And), aliasMap) val newAggregate = aggregate.copy(child = Filter(replaced, aggregate.child)) // If there is no more filter to stay up, just eliminate the filter. // Otherwise, create "Filter(stayUp) <- Aggregate <- Filter(pushDownPredicate)". @@ -1991,21 +1991,34 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe /** * Use [[With]] to rewrite condition which contains attribute that are not cheap and be consumed - * multiple times. + * multiple times. Each predicate generates one or 0 With. For facilitates subsequent merge + * [[With]], use the same CommonExpressionDef ids for different [[With]]. */ + private def rewriteConditionByWith( + cond: Seq[Expression], + aliasMap: AttributeMap[Alias]): Seq[Expression] = { + if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { + val canRewriteConf = cond.filter(canRewriteByWith) + if (canRewriteConf.nonEmpty) { + val replaceWithMap = canRewriteConf.reduce(And) + .collect { case a: Attribute => a } + .groupBy(identity) + .transform((_, v) => v.size) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => !CollapseProject.isCheap(m._2)) + val defsMap = AttributeMap(replaceWithMap.map(m => m._1 -> CommonExpressionDef(m._2))) + val refsMap = AttributeMap(defsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) + cond.map(rewriteByWith(_, defsMap, refsMap)) + } else cond + } else cond + } + private def rewriteConditionByWith( cond: Expression, aliasMap: AttributeMap[Alias]): Expression = { if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - val replaceWithMap = cond.collect { case a: Attribute => a } - .groupBy(identity) - .transform((_, v) => v.size) - .filter(m => aliasMap.contains(m._1) && m._2 > 1) - .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) - .filter(m => !CollapseProject.isCheap(m._2)) - splitConjunctivePredicates(cond) - .map(rewriteByWith(_, AttributeMap(replaceWithMap))) - .reduce(And) + rewriteConditionByWith(splitConjunctivePredicates(cond), aliasMap).reduce(And) } else cond } @@ -2016,16 +2029,20 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe private def rewriteByWith( expr: Expression, - replaceMap: AttributeMap[Expression]): Expression = { + defsMap: AttributeMap[CommonExpressionDef], + refsMap: AttributeMap[CommonExpressionRef]): Expression = { if (!canRewriteByWith(expr)) { return expr } - val exprAttrSet = expr.collect { case a: Attribute => a }.toSet - val newReplaceMap = replaceMap.filter(x => exprAttrSet.contains(x._1)) - if (newReplaceMap.isEmpty) { - return expr + val defs = mutable.HashSet.empty[CommonExpressionDef] + val replaced = expr.transform { + case a: Attribute if refsMap.contains(a) => + defs.add(defsMap.get(a).get) + refsMap.get(a).get } - With(expr, newReplaceMap) + if (defs.nonEmpty) { + With(replaced, defs.toSeq) + } else expr } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala index 6d53f3637dccb..1f5072934d04f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala @@ -96,7 +96,12 @@ object RewriteWithExpression extends Rule[LogicalPlan] { val defs = w.defs.map(rewriteWithExprAndInputPlans(_, inputPlans, isNestedWith = true)) val refToExpr = mutable.HashMap.empty[CommonExpressionId, Expression] val childProjections = Array.fill(inputPlans.length)(mutable.ArrayBuffer.empty[Alias]) - val refsCount = child.collect { case r: CommonExpressionRef => r} + val refsCount = child.collect { + case r: CommonExpressionRef + if defs.exists { + case d: CommonExpressionDef => d.id == r.id + } => r + } .groupBy(_.id) .transform((_, v) => v.size) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index 7b876eff1d950..c79e2abd29f6f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -156,22 +156,24 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("inner join with alias: alias contains multiple attributes") { - val t1 = testRelation.subquery("t1") - val t2 = testRelation.subquery("t2") - - val originalQuery = t1.select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") - .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) - .analyze - val correctAnswer = t1 - .select($"a", $"b", $"c", Coalesce(Seq($"a", $"b")) as "_common_expr_0") - .where(IsNotNull($"a") && IsNotNull($"_common_expr_0") && - $"a" === $"_common_expr_0") - .select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") - .join(t2.where(IsNotNull($"a")), Inner, - Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) - .analyze - val optimized = Optimize.execute(originalQuery) - comparePlans(optimized, correctAnswer) + withSQLConf(SQLConf.USE_COMMON_EXPR_ID_FOR_ALIAS.key -> "false") { + val t1 = testRelation.subquery("t1") + val t2 = testRelation.subquery("t2") + + val originalQuery = t1.select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") + .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) + .analyze + val correctAnswer = t1 + .select($"a", $"b", $"c", Coalesce(Seq($"a", $"b")) as "_common_expr_0") + .where(IsNotNull($"a") && IsNotNull($"_common_expr_0") && + $"a" === $"_common_expr_0") + .select($"a", Coalesce(Seq($"a", $"b")).as("int_col")).as("t") + .join(t2.where(IsNotNull($"a")), Inner, + Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) + .analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } } test("inner join with alias: alias contains single attributes") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 73f8f4be3a239..628f4bc2b9118 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -71,6 +71,11 @@ class SparkOptimizer( BooleanSimplification, PruneFilters), postHocOptimizationBatches, + // With expression will destroy extract python UDFs, so need rewrite it before. + Batch("Merge With expression", fixedPoint, MergeWithExpression), + Batch("Rewrite With expression", fixedPoint, + RewriteWithExpression, + CollapseProject), Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, // `ExtractPythonUDFFromJoinCondition` can convert a join to a cartesian product. diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index 87d0724ab7264..1329b3d9fb371 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -1,28 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Filter (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (8) - : : +- * Project (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet spark_catalog.default.item (4) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * Filter (16) - +- * ColumnarToRow (15) - +- Scan parquet spark_catalog.default.warehouse (14) +TakeOrderedAndProject (26) ++- * Project (25) + +- * Filter (24) + +- * Project (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.inventory (1) + : : +- BroadcastExchange (8) + : : +- * Project (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet spark_catalog.default.item (4) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * Filter (16) + +- * ColumnarToRow (15) + +- Scan parquet spark_catalog.default.warehouse (14) (1) Scan parquet spark_catalog.default.inventory @@ -72,7 +74,7 @@ Join condition: None Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_id#7] Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, i_item_id#7] -(11) ReusedExchange [Reuses operator id: 28] +(11) ReusedExchange [Reuses operator id: 30] Output [2]: [d_date_sk#9, d_date#10] (12) BroadcastHashJoin [codegen id : 4] @@ -131,38 +133,46 @@ Functions [2]: [sum(CASE WHEN (d_date#10 < 2000-03-11) THEN inv_quantity_on_hand Aggregate Attributes [2]: [sum(CASE WHEN (d_date#10 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#10 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] Results [4]: [w_warehouse_name#12, i_item_id#7, sum(CASE WHEN (d_date#10 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#10 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] -(23) Filter [codegen id : 5] +(23) Project [codegen id : 5] +Output [5]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20, CASE WHEN (inv_before#19 > 0) THEN (cast(inv_after#20 as double) / cast(inv_before#19 as double)) END AS _common_expr_0#21] Input [4]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20] -Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) -(24) TakeOrderedAndProject +(24) Filter [codegen id : 5] +Input [5]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20, _common_expr_0#21] +Condition : ((isnotnull(_common_expr_0#21) AND (_common_expr_0#21 >= 0.666667)) AND (_common_expr_0#21 <= 1.5)) + +(25) Project [codegen id : 5] +Output [4]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20] +Input [5]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20, _common_expr_0#21] + +(26) TakeOrderedAndProject Input [4]: [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20] Arguments: 100, [w_warehouse_name#12 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST], [w_warehouse_name#12, i_item_id#7, inv_before#19, inv_after#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * Filter (27) - +- * ColumnarToRow (26) - +- Scan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (30) ++- * Filter (29) + +- * ColumnarToRow (28) + +- Scan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(27) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_date#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_date#10] -(27) Filter [codegen id : 1] +(29) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_date#10] Condition : (((isnotnull(d_date#10) AND (d_date#10 >= 2000-02-10)) AND (d_date#10 <= 2000-04-10)) AND isnotnull(d_date_sk#9)) -(28) BroadcastExchange +(30) BroadcastExchange Input [2]: [d_date_sk#9, d_date#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt index 3b5e42e35739f..8e2c61787aa57 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt @@ -1,42 +1,44 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - InputAdapter - Exchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_warehouse_sk,inv_item_sk] - ColumnarToRow + Project [w_warehouse_name,i_item_id,inv_before,inv_after] + Filter [_common_expr_0] + Project [w_warehouse_name,i_item_id,inv_before,inv_after] + HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] + InputAdapter + Exchange [w_warehouse_name,i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_warehouse_sk,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #3 + WholeStageCodegen (1) + Project [i_item_sk,i_item_id] + Filter [i_current_price,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [i_item_sk,i_item_id] - Filter [i_current_price,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + ReusedExchange [d_date_sk,d_date] #2 InputAdapter - ReusedExchange [d_date_sk,d_date] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + WholeStageCodegen (3) + Filter [w_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 722be48a38eb1..57e98096d9fec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -1,28 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Filter (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) +TakeOrderedAndProject (26) ++- * Project (25) + +- * Filter (24) + +- * Project (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.inventory (1) + : : +- BroadcastExchange (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet spark_catalog.default.warehouse (4) + : +- BroadcastExchange (14) + : +- * Project (13) + : +- * Filter (12) + : +- * ColumnarToRow (11) + : +- Scan parquet spark_catalog.default.item (10) + +- ReusedExchange (17) (1) Scan parquet spark_catalog.default.inventory @@ -100,7 +102,7 @@ Join condition: None Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9] -(17) ReusedExchange [Reuses operator id: 28] +(17) ReusedExchange [Reuses operator id: 30] Output [2]: [d_date_sk#11, d_date#12] (18) BroadcastHashJoin [codegen id : 4] @@ -131,38 +133,46 @@ Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] -(23) Filter [codegen id : 5] +(23) Project [codegen id : 5] +Output [5]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20, CASE WHEN (inv_before#19 > 0) THEN (cast(inv_after#20 as double) / cast(inv_before#19 as double)) END AS _common_expr_0#21] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] -Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) -(24) TakeOrderedAndProject +(24) Filter [codegen id : 5] +Input [5]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20, _common_expr_0#21] +Condition : ((isnotnull(_common_expr_0#21) AND (_common_expr_0#21 >= 0.666667)) AND (_common_expr_0#21 <= 1.5)) + +(25) Project [codegen id : 5] +Output [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +Input [5]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20, _common_expr_0#21] + +(26) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * Filter (27) - +- * ColumnarToRow (26) - +- Scan parquet spark_catalog.default.date_dim (25) +BroadcastExchange (30) ++- * Filter (29) + +- * ColumnarToRow (28) + +- Scan parquet spark_catalog.default.date_dim (27) -(25) Scan parquet spark_catalog.default.date_dim +(27) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] -(27) Filter [codegen id : 1] +(29) Filter [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(28) BroadcastExchange +(30) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 09aab1082dcb3..1da39826f1536 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -1,42 +1,44 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - InputAdapter - Exchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_warehouse_sk,inv_item_sk] - ColumnarToRow + Project [w_warehouse_name,i_item_id,inv_before,inv_after] + Filter [_common_expr_0] + Project [w_warehouse_name,i_item_id,inv_before,inv_after] + HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] + InputAdapter + Exchange [w_warehouse_name,i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_warehouse_sk,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [w_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id] + Filter [i_current_price,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [i_item_sk,i_item_id] - Filter [i_current_price,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 + ReusedExchange [d_date_sk,d_date] #2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCBase.scala index 17645849225a6..517fcfe889a55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCBase.scala @@ -32,8 +32,10 @@ trait TPCBase extends SharedSparkSession { .set(SQLConf.CBO_ENABLED, true) .set(SQLConf.PLAN_STATS_ENABLED, true) .set(SQLConf.JOIN_REORDER_ENABLED, true) + .set(SQLConf.USE_COMMON_EXPR_ID_FOR_ALIAS, false) } else { super.sparkConf.set(SQLConf.MAX_TO_STRING_FIELDS, Int.MaxValue) + .set(SQLConf.USE_COMMON_EXPR_ID_FOR_ALIAS, false) } } From a94d51bb311a81e39cab9d7ac61a3043ef6df3aa Mon Sep 17 00:00:00 2001 From: zml1206 Date: Wed, 25 Dec 2024 13:44:41 +0800 Subject: [PATCH 15/18] fix --- .../approved-plans-v1_4/q9.sf100/explain.txt | 289 ++++++++---------- .../q9.sf100/simplified.txt | 105 +++---- .../approved-plans-v1_4/q9/explain.txt | 289 ++++++++---------- .../approved-plans-v1_4/q9/simplified.txt | 105 +++---- 4 files changed, 364 insertions(+), 424 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index e32db674082ee..f253fc3a02c72 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -26,14 +26,13 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* Project (12) -+- * HashAggregate (11) - +- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * Filter (7) - +- * ColumnarToRow (6) - +- Scan parquet spark_catalog.default.store_sales (5) +* HashAggregate (11) ++- Exchange (10) + +- * HashAggregate (9) + +- * Project (8) + +- * Filter (7) + +- * ColumnarToRow (6) + +- Scan parquet spark_catalog.default.store_sales (5) (5) Scan parquet spark_catalog.default.store_sales @@ -70,231 +69,207 @@ Input [5]: [count#26, sum#27, count#28, sum#29, count#30] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] Aggregate Attributes [3]: [count(1)#31, avg(UnscaledValue(ss_ext_discount_amt#18))#32, avg(UnscaledValue(ss_net_paid#19))#33] -Results [3]: [count(1)#31 AS count(1)#34, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#35, cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#36] - -(12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#34, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#35, avg(ss_net_paid), avg(ss_net_paid)#36) AS mergedValue#37] -Input [3]: [count(1)#34, avg(ss_ext_discount_amt)#35, avg(ss_net_paid)#36] +Results [1]: [named_struct(count(1), count(1)#31, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6))) AS mergedValue#34] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * Filter (15) - +- * ColumnarToRow (14) - +- Scan parquet spark_catalog.default.store_sales (13) - - -(13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +* HashAggregate (18) ++- Exchange (17) + +- * HashAggregate (16) + +- * Project (15) + +- * Filter (14) + +- * ColumnarToRow (13) + +- Scan parquet spark_catalog.default.store_sales (12) + + +(12) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +(13) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] -(15) Filter [codegen id : 1] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] -Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quantity#38 <= 40)) +(14) Filter [codegen id : 1] +Input [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#35) AND (ss_quantity#35 >= 21)) AND (ss_quantity#35 <= 40)) -(16) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +(15) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#36, ss_net_paid#37] +Input [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] -(17) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] +(16) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#36, ss_net_paid#37] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#39)), partial_avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [5]: [count#42, sum#43, count#44, sum#45, count#46] -Results [5]: [count#47, sum#48, count#49, sum#50, count#51] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#36)), partial_avg(UnscaledValue(ss_net_paid#37))] +Aggregate Attributes [5]: [count#39, sum#40, count#41, sum#42, count#43] +Results [5]: [count#44, sum#45, count#46, sum#47, count#48] -(18) Exchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +(17) Exchange +Input [5]: [count#44, sum#45, count#46, sum#47, count#48] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(19) HashAggregate [codegen id : 2] -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +(18) HashAggregate [codegen id : 2] +Input [5]: [count#44, sum#45, count#46, sum#47, count#48] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#39)), avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [3]: [count(1)#52, avg(UnscaledValue(ss_ext_discount_amt#39))#53, avg(UnscaledValue(ss_net_paid#40))#54] -Results [3]: [count(1)#52 AS count(1)#55, cast((avg(UnscaledValue(ss_ext_discount_amt#39))#53 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#56, cast((avg(UnscaledValue(ss_net_paid#40))#54 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#57] - -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#55, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#56, avg(ss_net_paid), avg(ss_net_paid)#57) AS mergedValue#58] -Input [3]: [count(1)#55, avg(ss_ext_discount_amt)#56, avg(ss_net_paid)#57] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#36)), avg(UnscaledValue(ss_net_paid#37))] +Aggregate Attributes [3]: [count(1)#49, avg(UnscaledValue(ss_ext_discount_amt#36))#50, avg(UnscaledValue(ss_net_paid#37))#51] +Results [1]: [named_struct(count(1), count(1)#49, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#36))#50 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#37))#51 / 100.0) as decimal(11,6))) AS mergedValue#52] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * Filter (23) - +- * ColumnarToRow (22) - +- Scan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +* HashAggregate (25) ++- Exchange (24) + +- * HashAggregate (23) + +- * Project (22) + +- * Filter (21) + +- * ColumnarToRow (20) + +- Scan parquet spark_catalog.default.store_sales (19) + + +(19) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct -(22) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +(20) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] -(23) Filter [codegen id : 1] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quantity#59 <= 60)) +(21) Filter [codegen id : 1] +Input [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#53) AND (ss_quantity#53 >= 41)) AND (ss_quantity#53 <= 60)) -(24) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +(22) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#54, ss_net_paid#55] +Input [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] -(25) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] +(23) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#54, ss_net_paid#55] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#60)), partial_avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [5]: [count#63, sum#64, count#65, sum#66, count#67] -Results [5]: [count#68, sum#69, count#70, sum#71, count#72] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#54)), partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [5]: [count#57, sum#58, count#59, sum#60, count#61] +Results [5]: [count#62, sum#63, count#64, sum#65, count#66] -(26) Exchange -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +(24) Exchange +Input [5]: [count#62, sum#63, count#64, sum#65, count#66] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 2] -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +(25) HashAggregate [codegen id : 2] +Input [5]: [count#62, sum#63, count#64, sum#65, count#66] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#60)), avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [3]: [count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#60))#74, avg(UnscaledValue(ss_net_paid#61))#75] -Results [3]: [count(1)#73 AS count(1)#76, cast((avg(UnscaledValue(ss_ext_discount_amt#60))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#77, cast((avg(UnscaledValue(ss_net_paid#61))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] - -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#76, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#77, avg(ss_net_paid), avg(ss_net_paid)#78) AS mergedValue#79] -Input [3]: [count(1)#76, avg(ss_ext_discount_amt)#77, avg(ss_net_paid)#78] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#54)), avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [3]: [count(1)#67, avg(UnscaledValue(ss_ext_discount_amt#54))#68, avg(UnscaledValue(ss_net_paid#55))#69] +Results [1]: [named_struct(count(1), count(1)#67, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#54))#68 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#55))#69 / 100.0) as decimal(11,6))) AS mergedValue#70] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.store_sales (29) - - -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +* HashAggregate (32) ++- Exchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * Filter (28) + +- * ColumnarToRow (27) + +- Scan parquet spark_catalog.default.store_sales (26) + + +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +(27) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] -(31) Filter [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quantity#80 <= 80)) +(28) Filter [codegen id : 1] +Input [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 61)) AND (ss_quantity#71 <= 80)) -(32) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +(29) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#72, ss_net_paid#73] +Input [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] -(33) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] +(30) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#72, ss_net_paid#73] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#81)), partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [5]: [count#84, sum#85, count#86, sum#87, count#88] -Results [5]: [count#89, sum#90, count#91, sum#92, count#93] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#72)), partial_avg(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [5]: [count#75, sum#76, count#77, sum#78, count#79] +Results [5]: [count#80, sum#81, count#82, sum#83, count#84] -(34) Exchange -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +(31) Exchange +Input [5]: [count#80, sum#81, count#82, sum#83, count#84] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 2] -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +(32) HashAggregate [codegen id : 2] +Input [5]: [count#80, sum#81, count#82, sum#83, count#84] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#81)), avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [3]: [count(1)#94, avg(UnscaledValue(ss_ext_discount_amt#81))#95, avg(UnscaledValue(ss_net_paid#82))#96] -Results [3]: [count(1)#94 AS count(1)#97, cast((avg(UnscaledValue(ss_ext_discount_amt#81))#95 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#98, cast((avg(UnscaledValue(ss_net_paid#82))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#99] - -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#97, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#98, avg(ss_net_paid), avg(ss_net_paid)#99) AS mergedValue#100] -Input [3]: [count(1)#97, avg(ss_ext_discount_amt)#98, avg(ss_net_paid)#99] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#72)), avg(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [3]: [count(1)#85, avg(UnscaledValue(ss_ext_discount_amt#72))#86, avg(UnscaledValue(ss_net_paid#73))#87] +Results [1]: [named_struct(count(1), count(1)#85, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#72))#86 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#73))#87 / 100.0) as decimal(11,6))) AS mergedValue#88] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +* HashAggregate (39) ++- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * Filter (35) + +- * ColumnarToRow (34) + +- Scan parquet spark_catalog.default.store_sales (33) + + +(33) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +(34) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] -(39) Filter [codegen id : 1] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_quantity#101 <= 100)) +(35) Filter [codegen id : 1] +Input [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#89) AND (ss_quantity#89 >= 81)) AND (ss_quantity#89 <= 100)) -(40) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +(36) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#90, ss_net_paid#91] +Input [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] +(37) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#90, ss_net_paid#91] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#102)), partial_avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [5]: [count#105, sum#106, count#107, sum#108, count#109] -Results [5]: [count#110, sum#111, count#112, sum#113, count#114] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#90)), partial_avg(UnscaledValue(ss_net_paid#91))] +Aggregate Attributes [5]: [count#93, sum#94, count#95, sum#96, count#97] +Results [5]: [count#98, sum#99, count#100, sum#101, count#102] -(42) Exchange -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +(38) Exchange +Input [5]: [count#98, sum#99, count#100, sum#101, count#102] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 2] -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +(39) HashAggregate [codegen id : 2] +Input [5]: [count#98, sum#99, count#100, sum#101, count#102] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#102)), avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [3]: [count(1)#115, avg(UnscaledValue(ss_ext_discount_amt#102))#116, avg(UnscaledValue(ss_net_paid#103))#117] -Results [3]: [count(1)#115 AS count(1)#118, cast((avg(UnscaledValue(ss_ext_discount_amt#102))#116 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#119, cast((avg(UnscaledValue(ss_net_paid#103))#117 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#120] - -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#118, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#119, avg(ss_net_paid), avg(ss_net_paid)#120) AS mergedValue#121] -Input [3]: [count(1)#118, avg(ss_ext_discount_amt)#119, avg(ss_net_paid)#120] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#90)), avg(UnscaledValue(ss_net_paid#91))] +Aggregate Attributes [3]: [count(1)#103, avg(UnscaledValue(ss_ext_discount_amt#90))#104, avg(UnscaledValue(ss_net_paid#91))#105] +Results [1]: [named_struct(count(1), count(1)#103, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#90))#104 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#91))#105 / 100.0) as decimal(11,6))) AS mergedValue#106] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt index 817fb0007f343..269aadd62e5b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt @@ -2,77 +2,72 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 Filter [r_reason_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index e32db674082ee..f253fc3a02c72 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -26,14 +26,13 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* Project (12) -+- * HashAggregate (11) - +- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * Filter (7) - +- * ColumnarToRow (6) - +- Scan parquet spark_catalog.default.store_sales (5) +* HashAggregate (11) ++- Exchange (10) + +- * HashAggregate (9) + +- * Project (8) + +- * Filter (7) + +- * ColumnarToRow (6) + +- Scan parquet spark_catalog.default.store_sales (5) (5) Scan parquet spark_catalog.default.store_sales @@ -70,231 +69,207 @@ Input [5]: [count#26, sum#27, count#28, sum#29, count#30] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] Aggregate Attributes [3]: [count(1)#31, avg(UnscaledValue(ss_ext_discount_amt#18))#32, avg(UnscaledValue(ss_net_paid#19))#33] -Results [3]: [count(1)#31 AS count(1)#34, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#35, cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#36] - -(12) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#34, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#35, avg(ss_net_paid), avg(ss_net_paid)#36) AS mergedValue#37] -Input [3]: [count(1)#34, avg(ss_ext_discount_amt)#35, avg(ss_net_paid)#36] +Results [1]: [named_struct(count(1), count(1)#31, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6))) AS mergedValue#34] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * Project (16) - +- * Filter (15) - +- * ColumnarToRow (14) - +- Scan parquet spark_catalog.default.store_sales (13) - - -(13) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +* HashAggregate (18) ++- Exchange (17) + +- * HashAggregate (16) + +- * Project (15) + +- * Filter (14) + +- * ColumnarToRow (13) + +- Scan parquet spark_catalog.default.store_sales (12) + + +(12) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +(13) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] -(15) Filter [codegen id : 1] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] -Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quantity#38 <= 40)) +(14) Filter [codegen id : 1] +Input [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#35) AND (ss_quantity#35 >= 21)) AND (ss_quantity#35 <= 40)) -(16) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +(15) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#36, ss_net_paid#37] +Input [4]: [ss_quantity#35, ss_ext_discount_amt#36, ss_net_paid#37, ss_sold_date_sk#38] -(17) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] +(16) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#36, ss_net_paid#37] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#39)), partial_avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [5]: [count#42, sum#43, count#44, sum#45, count#46] -Results [5]: [count#47, sum#48, count#49, sum#50, count#51] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#36)), partial_avg(UnscaledValue(ss_net_paid#37))] +Aggregate Attributes [5]: [count#39, sum#40, count#41, sum#42, count#43] +Results [5]: [count#44, sum#45, count#46, sum#47, count#48] -(18) Exchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +(17) Exchange +Input [5]: [count#44, sum#45, count#46, sum#47, count#48] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(19) HashAggregate [codegen id : 2] -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +(18) HashAggregate [codegen id : 2] +Input [5]: [count#44, sum#45, count#46, sum#47, count#48] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#39)), avg(UnscaledValue(ss_net_paid#40))] -Aggregate Attributes [3]: [count(1)#52, avg(UnscaledValue(ss_ext_discount_amt#39))#53, avg(UnscaledValue(ss_net_paid#40))#54] -Results [3]: [count(1)#52 AS count(1)#55, cast((avg(UnscaledValue(ss_ext_discount_amt#39))#53 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#56, cast((avg(UnscaledValue(ss_net_paid#40))#54 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#57] - -(20) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#55, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#56, avg(ss_net_paid), avg(ss_net_paid)#57) AS mergedValue#58] -Input [3]: [count(1)#55, avg(ss_ext_discount_amt)#56, avg(ss_net_paid)#57] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#36)), avg(UnscaledValue(ss_net_paid#37))] +Aggregate Attributes [3]: [count(1)#49, avg(UnscaledValue(ss_ext_discount_amt#36))#50, avg(UnscaledValue(ss_net_paid#37))#51] +Results [1]: [named_struct(count(1), count(1)#49, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#36))#50 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#37))#51 / 100.0) as decimal(11,6))) AS mergedValue#52] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * Filter (23) - +- * ColumnarToRow (22) - +- Scan parquet spark_catalog.default.store_sales (21) - - -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +* HashAggregate (25) ++- Exchange (24) + +- * HashAggregate (23) + +- * Project (22) + +- * Filter (21) + +- * ColumnarToRow (20) + +- Scan parquet spark_catalog.default.store_sales (19) + + +(19) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct -(22) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +(20) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] -(23) Filter [codegen id : 1] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quantity#59 <= 60)) +(21) Filter [codegen id : 1] +Input [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#53) AND (ss_quantity#53 >= 41)) AND (ss_quantity#53 <= 60)) -(24) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +(22) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#54, ss_net_paid#55] +Input [4]: [ss_quantity#53, ss_ext_discount_amt#54, ss_net_paid#55, ss_sold_date_sk#56] -(25) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] +(23) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#54, ss_net_paid#55] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#60)), partial_avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [5]: [count#63, sum#64, count#65, sum#66, count#67] -Results [5]: [count#68, sum#69, count#70, sum#71, count#72] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#54)), partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [5]: [count#57, sum#58, count#59, sum#60, count#61] +Results [5]: [count#62, sum#63, count#64, sum#65, count#66] -(26) Exchange -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +(24) Exchange +Input [5]: [count#62, sum#63, count#64, sum#65, count#66] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 2] -Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +(25) HashAggregate [codegen id : 2] +Input [5]: [count#62, sum#63, count#64, sum#65, count#66] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#60)), avg(UnscaledValue(ss_net_paid#61))] -Aggregate Attributes [3]: [count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#60))#74, avg(UnscaledValue(ss_net_paid#61))#75] -Results [3]: [count(1)#73 AS count(1)#76, cast((avg(UnscaledValue(ss_ext_discount_amt#60))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#77, cast((avg(UnscaledValue(ss_net_paid#61))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] - -(28) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#76, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#77, avg(ss_net_paid), avg(ss_net_paid)#78) AS mergedValue#79] -Input [3]: [count(1)#76, avg(ss_ext_discount_amt)#77, avg(ss_net_paid)#78] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#54)), avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [3]: [count(1)#67, avg(UnscaledValue(ss_ext_discount_amt#54))#68, avg(UnscaledValue(ss_net_paid#55))#69] +Results [1]: [named_struct(count(1), count(1)#67, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#54))#68 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#55))#69 / 100.0) as decimal(11,6))) AS mergedValue#70] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.store_sales (29) - - -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +* HashAggregate (32) ++- Exchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * Filter (28) + +- * ColumnarToRow (27) + +- Scan parquet spark_catalog.default.store_sales (26) + + +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +(27) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] -(31) Filter [codegen id : 1] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quantity#80 <= 80)) +(28) Filter [codegen id : 1] +Input [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 61)) AND (ss_quantity#71 <= 80)) -(32) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +(29) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#72, ss_net_paid#73] +Input [4]: [ss_quantity#71, ss_ext_discount_amt#72, ss_net_paid#73, ss_sold_date_sk#74] -(33) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] +(30) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#72, ss_net_paid#73] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#81)), partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [5]: [count#84, sum#85, count#86, sum#87, count#88] -Results [5]: [count#89, sum#90, count#91, sum#92, count#93] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#72)), partial_avg(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [5]: [count#75, sum#76, count#77, sum#78, count#79] +Results [5]: [count#80, sum#81, count#82, sum#83, count#84] -(34) Exchange -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +(31) Exchange +Input [5]: [count#80, sum#81, count#82, sum#83, count#84] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 2] -Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +(32) HashAggregate [codegen id : 2] +Input [5]: [count#80, sum#81, count#82, sum#83, count#84] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#81)), avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [3]: [count(1)#94, avg(UnscaledValue(ss_ext_discount_amt#81))#95, avg(UnscaledValue(ss_net_paid#82))#96] -Results [3]: [count(1)#94 AS count(1)#97, cast((avg(UnscaledValue(ss_ext_discount_amt#81))#95 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#98, cast((avg(UnscaledValue(ss_net_paid#82))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#99] - -(36) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#97, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#98, avg(ss_net_paid), avg(ss_net_paid)#99) AS mergedValue#100] -Input [3]: [count(1)#97, avg(ss_ext_discount_amt)#98, avg(ss_net_paid)#99] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#72)), avg(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [3]: [count(1)#85, avg(UnscaledValue(ss_ext_discount_amt#72))#86, avg(UnscaledValue(ss_net_paid#73))#87] +Results [1]: [named_struct(count(1), count(1)#85, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#72))#86 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#73))#87 / 100.0) as decimal(11,6))) AS mergedValue#88] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.store_sales (37) - - -(37) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +* HashAggregate (39) ++- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * Filter (35) + +- * ColumnarToRow (34) + +- Scan parquet spark_catalog.default.store_sales (33) + + +(33) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +(34) ColumnarToRow [codegen id : 1] +Input [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] -(39) Filter [codegen id : 1] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_quantity#101 <= 100)) +(35) Filter [codegen id : 1] +Input [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#89) AND (ss_quantity#89 >= 81)) AND (ss_quantity#89 <= 100)) -(40) Project [codegen id : 1] -Output [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +(36) Project [codegen id : 1] +Output [2]: [ss_ext_discount_amt#90, ss_net_paid#91] +Input [4]: [ss_quantity#89, ss_ext_discount_amt#90, ss_net_paid#91, ss_sold_date_sk#92] -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] +(37) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#90, ss_net_paid#91] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#102)), partial_avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [5]: [count#105, sum#106, count#107, sum#108, count#109] -Results [5]: [count#110, sum#111, count#112, sum#113, count#114] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#90)), partial_avg(UnscaledValue(ss_net_paid#91))] +Aggregate Attributes [5]: [count#93, sum#94, count#95, sum#96, count#97] +Results [5]: [count#98, sum#99, count#100, sum#101, count#102] -(42) Exchange -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +(38) Exchange +Input [5]: [count#98, sum#99, count#100, sum#101, count#102] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 2] -Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +(39) HashAggregate [codegen id : 2] +Input [5]: [count#98, sum#99, count#100, sum#101, count#102] Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#102)), avg(UnscaledValue(ss_net_paid#103))] -Aggregate Attributes [3]: [count(1)#115, avg(UnscaledValue(ss_ext_discount_amt#102))#116, avg(UnscaledValue(ss_net_paid#103))#117] -Results [3]: [count(1)#115 AS count(1)#118, cast((avg(UnscaledValue(ss_ext_discount_amt#102))#116 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#119, cast((avg(UnscaledValue(ss_net_paid#103))#117 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#120] - -(44) Project [codegen id : 2] -Output [1]: [named_struct(count(1), count(1)#118, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#119, avg(ss_net_paid), avg(ss_net_paid)#120) AS mergedValue#121] -Input [3]: [count(1)#118, avg(ss_ext_discount_amt)#119, avg(ss_net_paid)#120] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#90)), avg(UnscaledValue(ss_net_paid#91))] +Aggregate Attributes [3]: [count(1)#103, avg(UnscaledValue(ss_ext_discount_amt#90))#104, avg(UnscaledValue(ss_net_paid#91))#105] +Results [1]: [named_struct(count(1), count(1)#103, avg(ss_ext_discount_amt), cast((avg(UnscaledValue(ss_ext_discount_amt#90))#104 / 100.0) as decimal(11,6)), avg(ss_net_paid), cast((avg(UnscaledValue(ss_net_paid#91))#105 / 100.0) as decimal(11,6))) AS mergedValue#106] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 817fb0007f343..269aadd62e5b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -2,77 +2,72 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #1 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #2 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #3 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #4 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (2) - Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] - InputAdapter - Exchange #5 - WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] - Project [ss_ext_discount_amt,ss_net_paid] - Filter [ss_quantity] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),mergedValue,count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + Project [ss_ext_discount_amt,ss_net_paid] + Filter [ss_quantity] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 Filter [r_reason_sk] From 925f80eb285f43ee666288ec8f073b2c699ee8f9 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Wed, 25 Dec 2024 17:28:39 +0800 Subject: [PATCH 16/18] update --- .../sql/catalyst/optimizer/Optimizer.scala | 36 ++++++++----------- .../optimizer/RewriteWithExpression.scala | 6 ++-- 2 files changed, 17 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e0143a5bf225d..d1ebb2aa51eec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1998,19 +1998,16 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe cond: Seq[Expression], aliasMap: AttributeMap[Alias]): Seq[Expression] = { if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - val canRewriteConf = cond.filter(canRewriteByWith) - if (canRewriteConf.nonEmpty) { - val replaceWithMap = canRewriteConf.reduce(And) - .collect { case a: Attribute => a } - .groupBy(identity) - .transform((_, v) => v.size) - .filter(m => aliasMap.contains(m._1) && m._2 > 1) - .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) - .filter(m => !CollapseProject.isCheap(m._2)) - val defsMap = AttributeMap(replaceWithMap.map(m => m._1 -> CommonExpressionDef(m._2))) - val refsMap = AttributeMap(defsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) - cond.map(rewriteByWith(_, defsMap, refsMap)) - } else cond + val replaceWithMap = cond + .flatMap(_.collect {case a: Attribute => a }) + .groupBy(identity) + .transform((_, v) => v.size) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => needRewriteByWith(m._2)) + val defsMap = AttributeMap(replaceWithMap.map(m => m._1 -> CommonExpressionDef(m._2))) + val refsMap = AttributeMap(defsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) + cond.map(rewriteByWith(_, defsMap, refsMap)) } else cond } @@ -2022,23 +2019,20 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } else cond } - // With does not support inline subquery - private def canRewriteByWith(expr: Expression): Boolean = { - !expr.containsPattern(PLAN_EXPRESSION) + // With does not support common expression inline subquery. + private def needRewriteByWith(commonExpr: Expression): Boolean = { + !(commonExpr.containsPattern(PLAN_EXPRESSION) || CollapseProject.isCheap(commonExpr)) } private def rewriteByWith( expr: Expression, defsMap: AttributeMap[CommonExpressionDef], refsMap: AttributeMap[CommonExpressionRef]): Expression = { - if (!canRewriteByWith(expr)) { - return expr - } val defs = mutable.HashSet.empty[CommonExpressionDef] val replaced = expr.transform { case a: Attribute if refsMap.contains(a) => - defs.add(defsMap.get(a).get) - refsMap.get(a).get + defs.add(defsMap(a)) + refsMap(a) } if (defs.nonEmpty) { With(replaced, defs.toSeq) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala index 1f5072934d04f..7d1e13933e090 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala @@ -96,11 +96,9 @@ object RewriteWithExpression extends Rule[LogicalPlan] { val defs = w.defs.map(rewriteWithExprAndInputPlans(_, inputPlans, isNestedWith = true)) val refToExpr = mutable.HashMap.empty[CommonExpressionId, Expression] val childProjections = Array.fill(inputPlans.length)(mutable.ArrayBuffer.empty[Alias]) + val commonExprIdSet = defs.collect { case d: CommonExpressionDef => d.id }.toSet val refsCount = child.collect { - case r: CommonExpressionRef - if defs.exists { - case d: CommonExpressionDef => d.id == r.id - } => r + case r: CommonExpressionRef if commonExprIdSet.contains(r.id) => r } .groupBy(_.id) .transform((_, v) => v.size) From 9683b6841856f4b6a9b5ce93024f541809ad1130 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Wed, 25 Dec 2024 21:18:10 +0800 Subject: [PATCH 17/18] fix --- .../sql/catalyst/optimizer/Optimizer.scala | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d1ebb2aa51eec..167d638074de2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1998,16 +1998,19 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe cond: Seq[Expression], aliasMap: AttributeMap[Alias]): Seq[Expression] = { if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - val replaceWithMap = cond - .flatMap(_.collect {case a: Attribute => a }) - .groupBy(identity) - .transform((_, v) => v.size) - .filter(m => aliasMap.contains(m._1) && m._2 > 1) - .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) - .filter(m => needRewriteByWith(m._2)) - val defsMap = AttributeMap(replaceWithMap.map(m => m._1 -> CommonExpressionDef(m._2))) - val refsMap = AttributeMap(defsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) - cond.map(rewriteByWith(_, defsMap, refsMap)) + val canRewriteCond = cond.filter(canRewriteByWith) + if (canRewriteCond.nonEmpty) { + val replaceWithMap = canRewriteCond + .flatMap(_.collect {case a: Attribute => a }) + .groupBy(identity) + .transform((_, v) => v.size) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => !CollapseProject.isCheap(m._2)) + val defsMap = AttributeMap(replaceWithMap.map(m => m._1 -> CommonExpressionDef(m._2))) + val refsMap = AttributeMap(defsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) + cond.map(rewriteByWith(_, defsMap, refsMap)) + } else cond } else cond } @@ -2019,15 +2022,18 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } else cond } - // With does not support common expression inline subquery. - private def needRewriteByWith(commonExpr: Expression): Boolean = { - !(commonExpr.containsPattern(PLAN_EXPRESSION) || CollapseProject.isCheap(commonExpr)) + // With does not support inline subquery + private def canRewriteByWith(expr: Expression): Boolean = { + !expr.containsPattern(PLAN_EXPRESSION) } private def rewriteByWith( expr: Expression, defsMap: AttributeMap[CommonExpressionDef], refsMap: AttributeMap[CommonExpressionRef]): Expression = { + if (!canRewriteByWith(expr)) { + return expr + } val defs = mutable.HashSet.empty[CommonExpressionDef] val replaced = expr.transform { case a: Attribute if refsMap.contains(a) => From de6ede17e3cdf6044aec7758be238246aa123d3d Mon Sep 17 00:00:00 2001 From: zml1206 Date: Thu, 26 Dec 2024 16:49:21 +0800 Subject: [PATCH 18/18] optimize Subquery --- .../sql/catalyst/optimizer/Optimizer.scala | 60 +++++++++---------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 167d638074de2..7d0d933ae3fa4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1820,8 +1820,7 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe case Filter(condition, project @ Project(fields, grandChild)) if fields.forall(_.deterministic) && canPushThroughCondition(grandChild, condition) => val aliasMap = getAliasMap(project) - val replacedByWith = rewriteConditionByWith(condition, aliasMap) - project.copy(child = Filter(replaceAlias(replacedByWith, aliasMap), grandChild)) + project.copy(child = Filter(rewriteCondition(condition, aliasMap), grandChild)) // We can push down deterministic predicate through Aggregate, including throwable predicate. // If we can push down a filter through Aggregate, it means the filter only references the @@ -1841,8 +1840,7 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } if (pushDown.nonEmpty) { - val replacedByWith = rewriteConditionByWith(pushDown, aliasMap) - val replaced = replaceAlias(replacedByWith.reduce(And), aliasMap) + val replaced = rewriteCondition(pushDown.reduce(And), aliasMap) val newAggregate = aggregate.copy(child = Filter(replaced, aggregate.child)) // If there is no more filter to stay up, just eliminate the filter. // Otherwise, create "Filter(stayUp) <- Aggregate <- Filter(pushDownPredicate)". @@ -1989,51 +1987,53 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } } + private def rewriteCondition( + cond: Expression, + aliasMap: AttributeMap[Alias]): Expression = { + replaceAlias(rewriteConditionByWith(cond, aliasMap), aliasMap) + } + /** * Use [[With]] to rewrite condition which contains attribute that are not cheap and be consumed * multiple times. Each predicate generates one or 0 With. For facilitates subsequent merge * [[With]], use the same CommonExpressionDef ids for different [[With]]. */ private def rewriteConditionByWith( - cond: Seq[Expression], - aliasMap: AttributeMap[Alias]): Seq[Expression] = { + cond: Expression, + aliasMap: AttributeMap[Alias]): Expression = { if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - val canRewriteCond = cond.filter(canRewriteByWith) - if (canRewriteCond.nonEmpty) { - val replaceWithMap = canRewriteCond - .flatMap(_.collect {case a: Attribute => a }) - .groupBy(identity) - .transform((_, v) => v.size) - .filter(m => aliasMap.contains(m._1) && m._2 > 1) - .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) - .filter(m => !CollapseProject.isCheap(m._2)) + // SubqueryExpression can't contain common expression ref, replace alias for it first. + val newCond = replaceAliasForSubqueryExpression(cond, aliasMap) + val replaceWithMap = newCond.collect {case a: Attribute => a } + .groupBy(identity) + .transform((_, v) => v.size) + .filter(m => aliasMap.contains(m._1) && m._2 > 1) + .map(m => m._1 -> trimAliases(aliasMap.getOrElse(m._1, m._1))) + .filter(m => !CollapseProject.isCheap(m._2)) + if (replaceWithMap.isEmpty) { + newCond + } else { val defsMap = AttributeMap(replaceWithMap.map(m => m._1 -> CommonExpressionDef(m._2))) val refsMap = AttributeMap(defsMap.map(m => m._1 -> new CommonExpressionRef(m._2))) - cond.map(rewriteByWith(_, defsMap, refsMap)) - } else cond + splitConjunctivePredicates(newCond) + .map(rewriteByWith(_, defsMap, refsMap)) + .reduce(And) + } } else cond } - private def rewriteConditionByWith( - cond: Expression, + private def replaceAliasForSubqueryExpression( + expr: Expression, aliasMap: AttributeMap[Alias]): Expression = { - if (!SQLConf.get.getConf(SQLConf.ALWAYS_INLINE_COMMON_EXPR)) { - rewriteConditionByWith(splitConjunctivePredicates(cond), aliasMap).reduce(And) - } else cond - } - - // With does not support inline subquery - private def canRewriteByWith(expr: Expression): Boolean = { - !expr.containsPattern(PLAN_EXPRESSION) + expr.transform { + case s: SubqueryExpression => replaceAlias(s, aliasMap) + } } private def rewriteByWith( expr: Expression, defsMap: AttributeMap[CommonExpressionDef], refsMap: AttributeMap[CommonExpressionRef]): Expression = { - if (!canRewriteByWith(expr)) { - return expr - } val defs = mutable.HashSet.empty[CommonExpressionDef] val replaced = expr.transform { case a: Attribute if refsMap.contains(a) =>