From d5dda7468853bd1463c2db2f3c2eac8fa704aa25 Mon Sep 17 00:00:00 2001 From: ZhiHong Zhang Date: Wed, 6 Sep 2023 17:27:23 +0800 Subject: [PATCH] filter out equal to self, generate all possible predicates Why the plans of TPCDSV1_4_PlanStabilitySuite changed? For q5, the condition of EliminateOuterJoin only fulfilled after InferFiltersFromConstraints, so the added InferFiltersFromConstraints pushes more predicates down For q8, the added predicates of filter #19 is inferred by InferFiltersFromConstraints according to EqualNullSafe; about the added predicates of scan #13 and filter #15, after the predicate isnotnull(substr(ca_zip#9, 1, 5)) is pushed to filter #15 by current rules, then the laster InferFiltersFromConstraints pushes isnotnull(ca_zip#9) to filter #15 and scan #13. For q14a, the added predicates of filter #18 is inferred by InferFiltersFromConstraints according to EqualNullSafe. For q14b, it's same with q14a. For q85, PushExtraPredicateThroughJoin pushes predicates from join #21, then the added InferFiltersFromConstraints inferred the predicates from join #27 For q93, it's same as q5 Why the plans of TPCDSV2_7_PlanStabilityWithStatsSuite, TPCDSV1_4_PlanStabilityWithStatsSuite and TPCDSV2_7_PlanStabilitySuite changed? For TPCDSV2_7_PlanStabilityWithStatsSuite and TPCDSV2_7_PlanStabilitySuite, it's same as TPCDSV1_4_PlanStabilitySuite. For TPCDSV1_4_PlanStabilityWithStatsSuite, most plans are changed with same reason as TPCDSV1_4_PlanStabilitySuite, except q85, whose plan is changed further by rule CostBasedJoinReorder is triggered. --- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../plans/logical/QueryPlanConstraints.scala | 75 ++- .../optimizer/SetOperationSuite.scala | 2 +- .../analyzer-results/infer-filters.sql.out | 91 +++ .../sql-tests/inputs/infer-filters.sql | 38 ++ .../sql-tests/results/infer-filters.sql.out | 86 +++ .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14a/explain.txt | 4 +- .../approved-plans-v1_4/q14a/simplified.txt | 2 +- .../q14b.sf100/explain.txt | 4 +- .../q14b.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q14b/explain.txt | 4 +- .../approved-plans-v1_4/q14b/simplified.txt | 2 +- .../approved-plans-v1_4/q5.sf100/explain.txt | 150 ++--- .../q5.sf100/simplified.txt | 9 +- .../approved-plans-v1_4/q5/explain.txt | 140 +++-- .../approved-plans-v1_4/q5/simplified.txt | 9 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 7 +- .../q8.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q8/explain.txt | 7 +- .../approved-plans-v1_4/q8/simplified.txt | 2 +- .../approved-plans-v1_4/q85.sf100/explain.txt | 591 +++++++----------- .../q85.sf100/simplified.txt | 173 +++-- .../approved-plans-v1_4/q85/explain.txt | 4 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 46 +- .../q93.sf100/simplified.txt | 7 +- .../approved-plans-v1_4/q93/explain.txt | 98 +-- .../approved-plans-v1_4/q93/simplified.txt | 7 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 4 +- .../q14.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14/explain.txt | 4 +- .../approved-plans-v2_7/q14/simplified.txt | 2 +- .../q14a.sf100/explain.txt | 4 +- .../q14a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q14a/explain.txt | 4 +- .../approved-plans-v2_7/q14a/simplified.txt | 2 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 202 +++--- .../q5a.sf100/simplified.txt | 9 +- .../approved-plans-v2_7/q5a/explain.txt | 192 +++--- .../approved-plans-v2_7/q5a/simplified.txt | 9 +- 41 files changed, 1078 insertions(+), 928 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/infer-filters.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/infer-filters.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/infer-filters.sql.out 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 f533b150f0261..700ba6a527a6a 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 @@ -134,7 +134,7 @@ abstract class Optimizer(catalogManager: CatalogManager) val operatorOptimizationBatch: Seq[Batch] = { Batch("Operator Optimization before Inferring Filters", fixedPoint, operatorOptimizationRuleSet: _*) :: - Batch("Infer Filters", Once, + Batch("Infer Filters", fixedPoint, InferFiltersFromGenerate, InferFiltersFromConstraints) :: Batch("Operator Optimization after Inferring Filters", fixedPoint, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index 5a25ebadab77c..51db5c6e3e11d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -62,33 +62,60 @@ trait ConstraintHelper { */ def inferAdditionalConstraints(constraints: ExpressionSet): ExpressionSet = { var inferredConstraints = ExpressionSet() + + var lastInferredCount = 0 + var predicates = ExpressionSet() + var equalNullSafeCandidates = constraints.filter(_.deterministic) // IsNotNull should be constructed by `constructIsNotNullConstraints`. - val predicates = constraints.filterNot(_.isInstanceOf[IsNotNull]) - predicates.foreach { - case eq @ EqualTo(l: Attribute, r: Attribute) => - // Also remove EqualNullSafe with the same l and r to avoid Once strategy's idempotence - // is broken. l = r and l <=> r can infer l <=> l and r <=> r which is useless. - val candidateConstraints = predicates - eq - EqualNullSafe(l, r) - inferredConstraints ++= replaceConstraints(candidateConstraints, l, r) - inferredConstraints ++= replaceConstraints(candidateConstraints, r, l) - case eq @ EqualTo(l @ Cast(_: Attribute, _, _, _), r: Attribute) => - inferredConstraints ++= replaceConstraints(predicates - eq - EqualNullSafe(l, r), r, l) - case eq @ EqualTo(l: Attribute, r @ Cast(_: Attribute, _, _, _)) => - inferredConstraints ++= replaceConstraints(predicates - eq - EqualNullSafe(l, r), l, r) - case _ => // No inference - } + var equalToCandidates = equalNullSafeCandidates.filterNot(_.isInstanceOf[IsNotNull]) - constraints.foreach { - case eq @ EqualNullSafe(l: Attribute, r: Attribute) => - val candidateConstraints = constraints - eq - inferredConstraints ++= replaceConstraints(candidateConstraints, l, r) - inferredConstraints ++= replaceConstraints(candidateConstraints, r, l) - case eq @ EqualNullSafe(l @ Cast(_: Attribute, _, _, _), r: Attribute) => - inferredConstraints ++= replaceConstraints(constraints - eq, r, l) - case eq @ EqualNullSafe(l: Attribute, r @ Cast(_: Attribute, _, _, _)) => - inferredConstraints ++= replaceConstraints(constraints - eq, l, r) - case _ => // No inference + val refineEqualToCandidates = (curExp: Expression, l: Expression, r: Expression) => { + // Also remove EqualNullSafe with the same l and r, + // because l = r and l <=> r can infer l <=> l and r <=> r which are useless. + equalToCandidates - curExp - EqualNullSafe(l, r) - EqualNullSafe(r, l) + } + val refineEqualNullSafeCandidates = (curExp: Expression, l: Expression, r: Expression) => { + // Also remove EqualNullSafe and EqualTo with the same l and r, + // because l <=> r and l <=> r can infer l <=> l and r <=> r which are useless. + // l <=> r and l = r can infer l = l and r = r which is useless also. + equalNullSafeCandidates - curExp - EqualNullSafe(l, r) - EqualNullSafe(r, l) - + EqualTo(l, r) - EqualTo(r, l) } + + // For constrains such as [a = b, b = c, c = d], predicate a = d can not be inferred by just + // one iterator, so a while loop is needed to infer all possible predicates. + do { + lastInferredCount = inferredConstraints.size + + predicates = constraints.union(inferredConstraints) + predicates.foreach { + case eq @ EqualTo(l: Attribute, r: Attribute) => + val candidateConstraints = refineEqualToCandidates(eq, l, r) + inferredConstraints ++= replaceConstraints(candidateConstraints, l, r) + inferredConstraints ++= replaceConstraints(candidateConstraints, r, l) + case eq @ EqualTo(l @ Cast(_: Attribute, _, _, _), r: Attribute) => + inferredConstraints ++= replaceConstraints(refineEqualToCandidates(eq, l, r), r, l) + case eq @ EqualTo(l: Attribute, r @ Cast(_: Attribute, _, _, _)) => + inferredConstraints ++= replaceConstraints(refineEqualToCandidates(eq, l, r), l, r) + case _ => // No inference + } + + predicates.foreach { + case eq @ EqualNullSafe(l: Attribute, r: Attribute) => + val candidateConstraints = refineEqualNullSafeCandidates(eq, l, r) + inferredConstraints ++= replaceConstraints(candidateConstraints, l, r) + inferredConstraints ++= replaceConstraints(candidateConstraints, r, l) + case eq @ EqualNullSafe(l @ Cast(_: Attribute, _, _, _), r: Attribute) => + inferredConstraints ++= replaceConstraints(refineEqualNullSafeCandidates(eq, l, r), r, l) + case eq @ EqualNullSafe(l: Attribute, r @ Cast(_: Attribute, _, _, _)) => + inferredConstraints ++= replaceConstraints(refineEqualNullSafeCandidates(eq, l, r), l, r) + case _ => // No inference + } + + equalToCandidates = inferredConstraints.filterNot(_.isInstanceOf[IsNotNull]) + equalNullSafeCandidates = inferredConstraints + } while (inferredConstraints.size > lastInferredCount) + inferredConstraints -- constraints } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala index bd429c4f3636f..3b5c88b6428ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala @@ -394,7 +394,7 @@ class SetOperationSuite extends PlanTest { } - test("SPARK-44812: push filters through join generated from intersect") { + test("SPARK-44812: infer filters from join generated from intersect") { val a = LocalRelation.fromExternalRows(Seq($"a".int), Seq(Row(1), Row(2), Row(3))) val b = LocalRelation.fromExternalRows(Seq($"b".int), Seq(Row(2), Row(3), Row(4))) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/infer-filters.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/infer-filters.sql.out new file mode 100644 index 0000000000000..f6ed25b7165a0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/infer-filters.sql.out @@ -0,0 +1,91 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE i1_tbl ( + t1a integer, + t1b integer, + t1c integer, + t1d integer, + t1e integer, + t1f integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`i1_tbl`, false + + +-- !query +CREATE TABLE i2_tbl ( + t2a integer, + t2b integer, + t2c integer, + t2d integer, + t2e integer, + t2f integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`i2_tbl`, false + + +-- !query +CREATE TABLE i3_tbl ( + t3a integer, + t3b integer, + t3c integer, + t3d integer, + t3e integer, + t3f integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`i3_tbl`, false + + +-- !query +select i3_tbl.* from ( + select * from i1_tbl left semi join (select * from i2_tbl where t2a > 1000) tmp2 on i1_tbl.t1a = tmp2.t2a +) tmp1 join i3_tbl on tmp1.t1a = i3_tbl.t3a +-- !query analysis +Project [t3a#x, t3b#x, t3c#x, t3d#x, t3e#x, t3f#x] ++- Join Inner, (t1a#x = t3a#x) + :- SubqueryAlias tmp1 + : +- Project [t1a#x, t1b#x, t1c#x, t1d#x, t1e#x, t1f#x] + : +- Join LeftSemi, (t1a#x = t2a#x) + : :- SubqueryAlias spark_catalog.default.i1_tbl + : : +- Relation spark_catalog.default.i1_tbl[t1a#x,t1b#x,t1c#x,t1d#x,t1e#x,t1f#x] parquet + : +- SubqueryAlias tmp2 + : +- Project [t2a#x, t2b#x, t2c#x, t2d#x, t2e#x, t2f#x] + : +- Filter (t2a#x > 1000) + : +- SubqueryAlias spark_catalog.default.i2_tbl + : +- Relation spark_catalog.default.i2_tbl[t2a#x,t2b#x,t2c#x,t2d#x,t2e#x,t2f#x] parquet + +- SubqueryAlias spark_catalog.default.i3_tbl + +- Relation spark_catalog.default.i3_tbl[t3a#x,t3b#x,t3c#x,t3d#x,t3e#x,t3f#x] parquet + + +-- !query +select * from i1_tbl join i2_tbl on t1a = t2a and t2a = t1b and t1b = t2b and t2b = t1c and t1c = t2c and t2c = t1d and t1d = t2d and t2d = t1e and t1e = t2e and t2e = t1f +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#x, t1e#x, t1f#x, t2a#x, t2b#x, t2c#x, t2d#x, t2e#x, t2f#x] ++- Join Inner, (((((t1a#x = t2a#x) AND (t2a#x = t1b#x)) AND (t1b#x = t2b#x)) AND ((t2b#x = t1c#x) AND (t1c#x = t2c#x))) AND ((((t2c#x = t1d#x) AND (t1d#x = t2d#x)) AND (t2d#x = t1e#x)) AND ((t1e#x = t2e#x) AND (t2e#x = t1f#x)))) + :- SubqueryAlias spark_catalog.default.i1_tbl + : +- Relation spark_catalog.default.i1_tbl[t1a#x,t1b#x,t1c#x,t1d#x,t1e#x,t1f#x] parquet + +- SubqueryAlias spark_catalog.default.i2_tbl + +- Relation spark_catalog.default.i2_tbl[t2a#x,t2b#x,t2c#x,t2d#x,t2e#x,t2f#x] parquet + + +-- !query +DROP TABLE i1_tbl +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.i1_tbl + + +-- !query +DROP TABLE i2_tbl +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.i2_tbl + + +-- !query +DROP TABLE i3_tbl +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.i3_tbl diff --git a/sql/core/src/test/resources/sql-tests/inputs/infer-filters.sql b/sql/core/src/test/resources/sql-tests/inputs/infer-filters.sql new file mode 100644 index 0000000000000..5f4f481061b71 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/infer-filters.sql @@ -0,0 +1,38 @@ +CREATE TABLE i1_tbl ( + t1a integer, + t1b integer, + t1c integer, + t1d integer, + t1e integer, + t1f integer +) USING parquet; + +CREATE TABLE i2_tbl ( + t2a integer, + t2b integer, + t2c integer, + t2d integer, + t2e integer, + t2f integer +) USING parquet; + +CREATE TABLE i3_tbl ( + t3a integer, + t3b integer, + t3c integer, + t3d integer, + t3e integer, + t3f integer +) USING parquet; + + +select i3_tbl.* from ( + select * from i1_tbl left semi join (select * from i2_tbl where t2a > 1000) tmp2 on i1_tbl.t1a = tmp2.t2a +) tmp1 join i3_tbl on tmp1.t1a = i3_tbl.t3a; + + +select * from i1_tbl join i2_tbl on t1a = t2a and t2a = t1b and t1b = t2b and t2b = t1c and t1c = t2c and t2c = t1d and t1d = t2d and t2d = t1e and t1e = t2e and t2e = t1f; + +DROP TABLE i1_tbl; +DROP TABLE i2_tbl; +DROP TABLE i3_tbl; diff --git a/sql/core/src/test/resources/sql-tests/results/infer-filters.sql.out b/sql/core/src/test/resources/sql-tests/results/infer-filters.sql.out new file mode 100644 index 0000000000000..1773e1aca8a09 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/infer-filters.sql.out @@ -0,0 +1,86 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE i1_tbl ( + t1a integer, + t1b integer, + t1c integer, + t1d integer, + t1e integer, + t1f integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE i2_tbl ( + t2a integer, + t2b integer, + t2c integer, + t2d integer, + t2e integer, + t2f integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE i3_tbl ( + t3a integer, + t3b integer, + t3c integer, + t3d integer, + t3e integer, + t3f integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +select i3_tbl.* from ( + select * from i1_tbl left semi join (select * from i2_tbl where t2a > 1000) tmp2 on i1_tbl.t1a = tmp2.t2a +) tmp1 join i3_tbl on tmp1.t1a = i3_tbl.t3a +-- !query schema +struct +-- !query output + + + +-- !query +select * from i1_tbl join i2_tbl on t1a = t2a and t2a = t1b and t1b = t2b and t2b = t1c and t1c = t2c and t2c = t1d and t1d = t2d and t2d = t1e and t1e = t2e and t2e = t1f +-- !query schema +struct +-- !query output + + + +-- !query +DROP TABLE i1_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE i2_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE i3_tbl +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 8a53d4f895d3b..a115e02eacfef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -223,7 +223,7 @@ Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20] Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (25) ColumnarToRow [codegen id : 5] @@ -231,7 +231,7 @@ Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] (26) Filter [codegen id : 5] Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] -Condition : isnotnull(i_item_sk#21) +Condition : (((isnotnull(i_item_sk#21) AND isnotnull(i_brand_id#22)) AND isnotnull(i_class_id#23)) AND isnotnull(i_category_id#24)) (27) BroadcastExchange Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 7a7ce7f20b5b3..da5ce1c80da33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -145,7 +145,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #12 WholeStageCodegen (5) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 9c99ab3ad5b44..f17c03ec0ead8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -183,7 +183,7 @@ Condition : isnotnull(cs_item_sk#17) Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] @@ -191,7 +191,7 @@ Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Condition : (((isnotnull(i_item_sk#19) AND isnotnull(i_brand_id#20)) AND isnotnull(i_class_id#21)) AND isnotnull(i_category_id#22)) (19) BroadcastExchange Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index a047255e73dfd..c3a92d48dd9af 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -126,7 +126,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #10 WholeStageCodegen (1) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 0986e92088caa..23654bd684cac 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -202,7 +202,7 @@ Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20] Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (25) ColumnarToRow [codegen id : 5] @@ -210,7 +210,7 @@ Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] (26) Filter [codegen id : 5] Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] -Condition : isnotnull(i_item_sk#21) +Condition : (((isnotnull(i_item_sk#21) AND isnotnull(i_brand_id#22)) AND isnotnull(i_class_id#23)) AND isnotnull(i_category_id#24)) (27) BroadcastExchange Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 4db035858b224..60178a502526d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -144,7 +144,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #11 WholeStageCodegen (5) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 3f4f3653371d9..9444cf61c5797 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -162,7 +162,7 @@ Condition : isnotnull(cs_item_sk#17) Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] @@ -170,7 +170,7 @@ Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Condition : (((isnotnull(i_item_sk#19) AND isnotnull(i_brand_id#20)) AND isnotnull(i_class_id#21)) AND isnotnull(i_category_id#22)) (19) BroadcastExchange Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index b103e79d11894..6d70272249736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -125,7 +125,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 6bc3c6747b31c..14ec7db1a8b8e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Expand (73) - +- Union (72) +TakeOrderedAndProject (78) ++- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * Expand (74) + +- Union (73) :- * HashAggregate (21) : +- Exchange (20) : +- * HashAggregate (19) @@ -47,35 +47,36 @@ TakeOrderedAndProject (77) : : +- * ColumnarToRow (32) : : +- Scan parquet spark_catalog.default.catalog_page (31) : +- ReusedExchange (37) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- Union (59) + +- * HashAggregate (72) + +- Exchange (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- Union (60) : : :- * Project (46) : : : +- * Filter (45) : : : +- * ColumnarToRow (44) : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : +- * Project (58) - : : +- * SortMergeJoin Inner (57) - : : :- * Sort (50) - : : : +- Exchange (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : +- * Sort (56) - : : +- Exchange (55) - : : +- * Project (54) - : : +- * Filter (53) - : : +- * ColumnarToRow (52) - : : +- Scan parquet spark_catalog.default.web_sales (51) - : +- BroadcastExchange (63) - : +- * Filter (62) - : +- * ColumnarToRow (61) - : +- Scan parquet spark_catalog.default.web_site (60) - +- ReusedExchange (66) + : : +- * Project (59) + : : +- * SortMergeJoin Inner (58) + : : :- * Sort (51) + : : : +- Exchange (50) + : : : +- * Filter (49) + : : : +- * ColumnarToRow (48) + : : : +- Scan parquet spark_catalog.default.web_returns (47) + : : +- * Sort (57) + : : +- Exchange (56) + : : +- * Project (55) + : : +- * Filter (54) + : : +- * ColumnarToRow (53) + : : +- Scan parquet spark_catalog.default.web_sales (52) + : +- BroadcastExchange (64) + : +- * Filter (63) + : +- * ColumnarToRow (62) + : +- Scan parquet spark_catalog.default.web_site (61) + +- ReusedExchange (67) (1) Scan parquet spark_catalog.default.store_sales @@ -146,7 +147,7 @@ Join condition: None Output [6]: [date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#23] Input [8]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#22, s_store_id#23] -(16) ReusedExchange [Reuses operator id: 82] +(16) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#24] (17) BroadcastHashJoin [codegen id : 5] @@ -245,7 +246,7 @@ Join condition: None Output [6]: [date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#63] Input [8]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_sk#62, cp_catalog_page_id#63] -(37) ReusedExchange [Reuses operator id: 82] +(37) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#64] (38) BroadcastHashJoin [codegen id : 11] @@ -300,173 +301,178 @@ Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (48) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -(49) Exchange +(49) Filter [codegen id : 14] +Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +Condition : (isnotnull(wr_item_sk#92) AND isnotnull(wr_order_number#93)) + +(50) Exchange Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] Arguments: hashpartitioning(wr_item_sk#92, wr_order_number#93, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) Sort [codegen id : 15] +(51) Sort [codegen id : 15] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] Arguments: [wr_item_sk#92 ASC NULLS FIRST, wr_order_number#93 ASC NULLS FIRST], false, 0 -(51) Scan parquet spark_catalog.default.web_sales +(52) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 16] +(53) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(53) Filter [codegen id : 16] +(54) Filter [codegen id : 16] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) -(54) Project [codegen id : 16] +(55) Project [codegen id : 16] Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(55) Exchange +(56) Exchange Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Arguments: hashpartitioning(ws_item_sk#97, ws_order_number#99, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(56) Sort [codegen id : 17] +(57) Sort [codegen id : 17] Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Arguments: [ws_item_sk#97 ASC NULLS FIRST, ws_order_number#99 ASC NULLS FIRST], false, 0 -(57) SortMergeJoin [codegen id : 18] +(58) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#92, wr_order_number#93] Right keys [2]: [ws_item_sk#97, ws_order_number#99] Join type: Inner Join condition: None -(58) Project [codegen id : 18] +(59) Project [codegen id : 18] Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -(59) Union +(60) Union -(60) Scan parquet spark_catalog.default.web_site +(61) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#107, web_site_id#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 19] +(62) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] -(62) Filter [codegen id : 19] +(63) Filter [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] Condition : isnotnull(web_site_sk#107) -(63) BroadcastExchange +(64) BroadcastExchange Input [2]: [web_site_sk#107, web_site_id#108] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(64) BroadcastHashJoin [codegen id : 21] +(65) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#86] Right keys [1]: [web_site_sk#107] Join type: Inner Join condition: None -(65) Project [codegen id : 21] +(66) Project [codegen id : 21] Output [6]: [date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108] Input [8]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#107, web_site_id#108] -(66) ReusedExchange [Reuses operator id: 82] +(67) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#109] -(67) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(68) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108] Input [7]: [date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108, d_date_sk#109] -(69) HashAggregate [codegen id : 21] +(70) HashAggregate [codegen id : 21] Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108] Keys [1]: [web_site_id#108] Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] Results [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117] -(70) Exchange +(71) Exchange Input [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117] Arguments: hashpartitioning(web_site_id#108, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(71) HashAggregate [codegen id : 22] +(72) HashAggregate [codegen id : 22] Input [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117] Keys [1]: [web_site_id#108] Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#108) AS id#126] -(72) Union +(73) Union -(73) Expand [codegen id : 23] +(74) Expand [codegen id : 23] Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] -(74) HashAggregate [codegen id : 23] +(75) HashAggregate [codegen id : 23] Input [6]: [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] Keys [3]: [channel#127, id#128, spark_grouping_id#129] Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] Results [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(75) Exchange +(76) Exchange Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Arguments: hashpartitioning(channel#127, id#128, spark_grouping_id#129, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(76) HashAggregate [codegen id : 24] +(77) HashAggregate [codegen id : 24] Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [3]: [channel#127, id#128, spark_grouping_id#129] Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] Aggregate Attributes [3]: [sum(sales#37)#142, sum(returns#38)#143, sum(profit#39)#144] Results [5]: [channel#127, id#128, sum(sales#37)#142 AS sales#145, sum(returns#38)#143 AS returns#146, sum(profit#39)#144 AS profit#147] -(77) TakeOrderedAndProject +(78) TakeOrderedAndProject Input [5]: [channel#127, id#128, sales#145, returns#146, profit#147] Arguments: 100, [channel#127 ASC NULLS FIRST, id#128 ASC NULLS FIRST], [channel#127, id#128, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (82) -+- * Project (81) - +- * Filter (80) - +- * ColumnarToRow (79) - +- Scan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (83) ++- * Project (82) + +- * Filter (81) + +- * ColumnarToRow (80) + +- Scan parquet spark_catalog.default.date_dim (79) -(78) Scan parquet spark_catalog.default.date_dim +(79) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#148] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 1] +(80) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_date#148] -(80) Filter [codegen id : 1] +(81) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_date#148] Condition : (((isnotnull(d_date#148) AND (d_date#148 >= 2000-08-23)) AND (d_date#148 <= 2000-09-06)) AND isnotnull(d_date_sk#24)) -(81) Project [codegen id : 1] +(82) Project [codegen id : 1] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#148] -(82) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index 2547db4f5e5fb..8395c291e8365 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -113,10 +113,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 313959456c809..2d2c4c19891a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Expand (70) - +- Union (69) +TakeOrderedAndProject (75) ++- * HashAggregate (74) + +- Exchange (73) + +- * HashAggregate (72) + +- * Expand (71) + +- Union (70) :- * HashAggregate (21) : +- Exchange (20) : +- * HashAggregate (19) @@ -47,32 +47,33 @@ TakeOrderedAndProject (74) : +- * Filter (36) : +- * ColumnarToRow (35) : +- Scan parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- Union (56) + +- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- Union (57) : : :- * Project (46) : : : +- * Filter (45) : : : +- * ColumnarToRow (44) : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : +- * Project (55) - : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : :- BroadcastExchange (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : +- * Project (53) - : : +- * Filter (52) - : : +- * ColumnarToRow (51) - : : +- Scan parquet spark_catalog.default.web_sales (50) - : +- ReusedExchange (57) - +- BroadcastExchange (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet spark_catalog.default.web_site (60) + : : +- * Project (56) + : : +- * BroadcastHashJoin Inner BuildLeft (55) + : : :- BroadcastExchange (50) + : : : +- * Filter (49) + : : : +- * ColumnarToRow (48) + : : : +- Scan parquet spark_catalog.default.web_returns (47) + : : +- * Project (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet spark_catalog.default.web_sales (51) + : +- ReusedExchange (58) + +- BroadcastExchange (64) + +- * Filter (63) + +- * ColumnarToRow (62) + +- Scan parquet spark_catalog.default.web_site (61) (1) Scan parquet spark_catalog.default.store_sales @@ -115,7 +116,7 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (9) Union -(10) ReusedExchange [Reuses operator id: 79] +(10) ReusedExchange [Reuses operator id: 80] Output [1]: [d_date_sk#22] (11) BroadcastHashJoin [codegen id : 5] @@ -214,7 +215,7 @@ Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_retur (30) Union -(31) ReusedExchange [Reuses operator id: 79] +(31) ReusedExchange [Reuses operator id: 80] Output [1]: [d_date_sk#62] (32) BroadcastHashJoin [codegen id : 11] @@ -297,161 +298,166 @@ Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (48) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -(49) BroadcastExchange +(49) Filter [codegen id : 14] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Condition : (isnotnull(wr_item_sk#92) AND isnotnull(wr_order_number#93)) -(50) Scan parquet spark_catalog.default.web_sales +(50) BroadcastExchange +Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=5] + +(51) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(51) ColumnarToRow +(52) ColumnarToRow Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(52) Filter +(53) Filter Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) -(53) Project +(54) Project Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(54) BroadcastHashJoin [codegen id : 15] +(55) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#92, wr_order_number#93] Right keys [2]: [ws_item_sk#97, ws_order_number#99] Join type: Inner Join condition: None -(55) Project [codegen id : 15] +(56) Project [codegen id : 15] Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -(56) Union +(57) Union -(57) ReusedExchange [Reuses operator id: 79] +(58) ReusedExchange [Reuses operator id: 80] Output [1]: [d_date_sk#107] -(58) BroadcastHashJoin [codegen id : 18] +(59) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(59) Project [codegen id : 18] +(60) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91] Input [7]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, d_date_sk#107] -(60) Scan parquet spark_catalog.default.web_site +(61) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#108, web_site_id#109] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 17] +(62) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] -(62) Filter [codegen id : 17] +(63) Filter [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] Condition : isnotnull(web_site_sk#108) -(63) BroadcastExchange +(64) BroadcastExchange Input [2]: [web_site_sk#108, web_site_id#109] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(64) BroadcastHashJoin [codegen id : 18] +(65) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#86] Right keys [1]: [web_site_sk#108] Join type: Inner Join condition: None -(65) Project [codegen id : 18] +(66) Project [codegen id : 18] Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] Input [7]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#108, web_site_id#109] -(66) HashAggregate [codegen id : 18] +(67) HashAggregate [codegen id : 18] Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] Keys [1]: [web_site_id#109] Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] Results [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -(67) Exchange +(68) Exchange Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] Arguments: hashpartitioning(web_site_id#109, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(68) HashAggregate [codegen id : 19] +(69) HashAggregate [codegen id : 19] Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] Keys [1]: [web_site_id#109] Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#109) AS id#126] -(69) Union +(70) Union -(70) Expand [codegen id : 20] +(71) Expand [codegen id : 20] Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] -(71) HashAggregate [codegen id : 20] +(72) HashAggregate [codegen id : 20] Input [6]: [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] Keys [3]: [channel#127, id#128, spark_grouping_id#129] Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] Results [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(72) Exchange +(73) Exchange Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Arguments: hashpartitioning(channel#127, id#128, spark_grouping_id#129, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(73) HashAggregate [codegen id : 21] +(74) HashAggregate [codegen id : 21] Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [3]: [channel#127, id#128, spark_grouping_id#129] Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] Aggregate Attributes [3]: [sum(sales#37)#142, sum(returns#38)#143, sum(profit#39)#144] Results [5]: [channel#127, id#128, sum(sales#37)#142 AS sales#145, sum(returns#38)#143 AS returns#146, sum(profit#39)#144 AS profit#147] -(74) TakeOrderedAndProject +(75) TakeOrderedAndProject Input [5]: [channel#127, id#128, sales#145, returns#146, profit#147] Arguments: 100, [channel#127 ASC NULLS FIRST, id#128 ASC NULLS FIRST], [channel#127, id#128, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * Project (78) - +- * Filter (77) - +- * ColumnarToRow (76) - +- Scan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (80) ++- * Project (79) + +- * Filter (78) + +- * ColumnarToRow (77) + +- Scan parquet spark_catalog.default.date_dim (76) -(75) Scan parquet spark_catalog.default.date_dim +(76) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#148] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 1] +(77) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_date#148] -(77) Filter [codegen id : 1] +(78) Filter [codegen id : 1] Input [2]: [d_date_sk#22, d_date#148] Condition : (((isnotnull(d_date#148) AND (d_date#148 >= 2000-08-23)) AND (d_date#148 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(78) Project [codegen id : 1] +(79) Project [codegen id : 1] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#148] -(79) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index c6cd47e3f48ef..476811b554039 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -110,10 +110,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index d443723b06386..ec41a8f9ff763 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -118,6 +118,7 @@ Arguments: [substr(s_zip#8, 1, 2) ASC NULLS FIRST], false, 0 Output [1]: [ca_zip#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_zip)] ReadSchema: struct (16) ColumnarToRow [codegen id : 11] @@ -125,7 +126,7 @@ Input [1]: [ca_zip#9] (17) Filter [codegen id : 11] Input [1]: [ca_zip#9] -Condition : (substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#9, 1, 5))) +Condition : ((isnotnull(ca_zip#9) AND substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543) AND isnotnull(substr(ca_zip#9, 1, 5))) (18) Project [codegen id : 11] Output [1]: [substr(ca_zip#9, 1, 5) AS ca_zip#10] @@ -135,7 +136,7 @@ Input [1]: [ca_zip#9] Output [2]: [ca_address_sk#11, ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] +PushedFilters: [IsNotNull(ca_zip), IsNotNull(ca_address_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 5] @@ -143,7 +144,7 @@ Input [2]: [ca_address_sk#11, ca_zip#12] (21) Filter [codegen id : 5] Input [2]: [ca_address_sk#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#11) +Condition : (((isnotnull(ca_zip#12) AND isnotnull(ca_address_sk#11)) AND substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543) AND isnotnull(substr(ca_zip#12, 1, 5))) (22) Exchange Input [2]: [ca_address_sk#11, ca_zip#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index 86dd6134fc27f..f033f4b3b73fb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -73,7 +73,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] InputAdapter Exchange [ca_address_sk] #9 WholeStageCodegen (5) - Filter [ca_address_sk] + Filter [ca_zip,ca_address_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 24c3a657ddd20..0917179b62076 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -104,6 +104,7 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip# Output [1]: [ca_zip#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_zip)] ReadSchema: struct (14) ColumnarToRow [codegen id : 6] @@ -111,7 +112,7 @@ Input [1]: [ca_zip#9] (15) Filter [codegen id : 6] Input [1]: [ca_zip#9] -Condition : (substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#9, 1, 5))) +Condition : ((isnotnull(ca_zip#9) AND substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543) AND isnotnull(substr(ca_zip#9, 1, 5))) (16) Project [codegen id : 6] Output [1]: [substr(ca_zip#9, 1, 5) AS ca_zip#10] @@ -121,7 +122,7 @@ Input [1]: [ca_zip#9] Output [2]: [ca_address_sk#11, ca_zip#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] +PushedFilters: [IsNotNull(ca_zip), IsNotNull(ca_address_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 4] @@ -129,7 +130,7 @@ Input [2]: [ca_address_sk#11, ca_zip#12] (19) Filter [codegen id : 4] Input [2]: [ca_address_sk#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#11) +Condition : (((isnotnull(ca_zip#12) AND isnotnull(ca_address_sk#11)) AND substr(ca_zip#12, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543) AND isnotnull(substr(ca_zip#12, 1, 5))) (20) Scan parquet spark_catalog.default.customer Output [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 6ea5a786125cc..51bdb89a7b65d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -58,7 +58,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] HashAggregate [ca_zip] [count,count] Project [ca_zip] BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - Filter [ca_address_sk] + Filter [ca_zip,ca_address_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 410a6a1957505..f447fac3d29bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -1,432 +1,325 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (41) - : : +- * BroadcastHashJoin Inner BuildRight (40) - : : :- * Project (34) - : : : +- * SortMergeJoin Inner (33) - : : : :- * Sort (27) - : : : : +- Exchange (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (19) - : : : : : +- * SortMergeJoin Inner (18) - : : : : : :- * Sort (11) - : : : : : : +- Exchange (10) - : : : : : : +- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.web_page (4) - : : : : : +- * Sort (17) - : : : : : +- Exchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet spark_catalog.default.web_returns (12) - : : : : +- BroadcastExchange (23) - : : : : +- * Filter (22) - : : : : +- * ColumnarToRow (21) - : : : : +- Scan parquet spark_catalog.default.customer_demographics (20) - : : : +- * Sort (32) - : : : +- Exchange (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet spark_catalog.default.customer_demographics (28) - : : +- BroadcastExchange (39) - : : +- * Project (38) - : : +- * Filter (37) - : : +- * ColumnarToRow (36) - : : +- Scan parquet spark_catalog.default.customer_address (35) - : +- ReusedExchange (42) - +- BroadcastExchange (48) - +- * Filter (47) - +- * ColumnarToRow (46) - +- Scan parquet spark_catalog.default.reason (45) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * SortMergeJoin Inner (37) + : : :- * Sort (31) + : : : +- Exchange (30) + : : : +- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (22) + : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : :- * Project (16) + : : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Project (4) + : : : : : : : +- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet spark_catalog.default.web_returns (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Filter (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- Scan parquet spark_catalog.default.customer_demographics (5) + : : : : : +- BroadcastExchange (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet spark_catalog.default.customer_demographics (11) + : : : : +- BroadcastExchange (20) + : : : : +- * Filter (19) + : : : : +- * ColumnarToRow (18) + : : : : +- Scan parquet spark_catalog.default.reason (17) + : : : +- BroadcastExchange (27) + : : : +- * Project (26) + : : : +- * Filter (25) + : : : +- * ColumnarToRow (24) + : : : +- Scan parquet spark_catalog.default.customer_address (23) + : : +- * Sort (36) + : : +- Exchange (35) + : : +- * Filter (34) + : : +- * ColumnarToRow (33) + : : +- Scan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (39) + +- BroadcastExchange (45) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet spark_catalog.default.web_page (42) + + +(1) Scan parquet spark_catalog.default.web_returns +Output [9]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_returning_cdemo_sk#4, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, wr_returned_date_sk#9] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 5] +Input [9]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_returning_cdemo_sk#4, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, wr_returned_date_sk#9] -(2) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +(3) Filter [codegen id : 5] +Input [9]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_returning_cdemo_sk#4, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, wr_returned_date_sk#9] +Condition : (((((isnotnull(wr_item_sk#1) AND isnotnull(wr_order_number#6)) AND isnotnull(wr_refunded_cdemo_sk#2)) AND isnotnull(wr_returning_cdemo_sk#4)) AND isnotnull(wr_refunded_addr_sk#3)) AND isnotnull(wr_reason_sk#5)) -(3) Filter [codegen id : 2] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) +(4) Project [codegen id : 5] +Output [8]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_returning_cdemo_sk#4, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8] +Input [9]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_returning_cdemo_sk#4, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, wr_returned_date_sk#9] -(4) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#9] +(5) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [1]: [wp_web_page_sk#9] +(6) ColumnarToRow [codegen id : 1] +Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -(6) Filter [codegen id : 1] -Input [1]: [wp_web_page_sk#9] -Condition : isnotnull(wp_web_page_sk#9) +(7) Filter [codegen id : 1] +Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Condition : (((isnotnull(cd_demo_sk#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND ((((cd_marital_status#11 = M) AND (cd_education_status#12 = Advanced Degree )) OR ((cd_marital_status#11 = S) AND (cd_education_status#12 = College ))) OR ((cd_marital_status#11 = W) AND (cd_education_status#12 = 2 yr Degree )))) -(7) BroadcastExchange -Input [1]: [wp_web_page_sk#9] +(8) BroadcastExchange +Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_web_page_sk#2] -Right keys [1]: [wp_web_page_sk#9] +(9) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [wr_returning_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(9) Project [codegen id : 2] -Output [6]: [ws_item_sk#1, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Input [8]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wp_web_page_sk#9] +(10) Project [codegen id : 5] +Output [9]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#11, cd_education_status#12] +Input [11]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_returning_cdemo_sk#4, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -(10) Exchange -Input [6]: [ws_item_sk#1, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: hashpartitioning(ws_item_sk#1, ws_order_number#3, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(11) Sort [codegen id : 3] -Input [6]: [ws_item_sk#1, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, ws_order_number#3 ASC NULLS FIRST], false, 0 - -(12) Scan parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17, wr_returned_date_sk#18] +(11) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 4] -Input [9]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17, wr_returned_date_sk#18] - -(14) Filter [codegen id : 4] -Input [9]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17, wr_returned_date_sk#18] -Condition : (((((((isnotnull(wr_item_sk#10) AND isnotnull(wr_order_number#15)) AND isnotnull(wr_refunded_cdemo_sk#11)) AND isnotnull(wr_returning_cdemo_sk#13)) AND isnotnull(wr_refunded_addr_sk#12)) AND isnotnull(wr_reason_sk#14)) AND might_contain(Subquery scalar-subquery#19, [id=#20], xxhash64(wr_refunded_cdemo_sk#11, 42))) AND might_contain(Subquery scalar-subquery#21, [id=#22], xxhash64(wr_refunded_addr_sk#12, 42))) +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +ReadSchema: struct -(15) Project [codegen id : 4] -Output [8]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17] -Input [9]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17, wr_returned_date_sk#18] +(12) ColumnarToRow [codegen id : 2] +Input [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] -(16) Exchange -Input [8]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17] -Arguments: hashpartitioning(wr_item_sk#10, wr_order_number#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(13) Filter [codegen id : 2] +Input [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] +Condition : (((isnotnull(cd_demo_sk#13) AND isnotnull(cd_marital_status#14)) AND isnotnull(cd_education_status#15)) AND ((((cd_marital_status#14 = M) AND (cd_education_status#15 = Advanced Degree )) OR ((cd_marital_status#14 = S) AND (cd_education_status#15 = College ))) OR ((cd_marital_status#14 = W) AND (cd_education_status#15 = 2 yr Degree )))) -(17) Sort [codegen id : 5] -Input [8]: [wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17] -Arguments: [wr_item_sk#10 ASC NULLS FIRST, wr_order_number#15 ASC NULLS FIRST], false, 0 +(14) BroadcastExchange +Input [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], input[1, string, false], input[2, string, false]),false), [plan_id=2] -(18) SortMergeJoin [codegen id : 7] -Left keys [2]: [ws_item_sk#1, ws_order_number#3] -Right keys [2]: [wr_item_sk#10, wr_order_number#15] +(15) BroadcastHashJoin [codegen id : 5] +Left keys [3]: [wr_refunded_cdemo_sk#2, cd_marital_status#11, cd_education_status#12] +Right keys [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] Join type: Inner Join condition: None -(19) Project [codegen id : 7] -Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17] -Input [14]: [ws_item_sk#1, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#10, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_order_number#15, wr_fee#16, wr_refunded_cash#17] +(16) Project [codegen id : 5] +Output [8]: [wr_item_sk#1, wr_refunded_addr_sk#3, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15] +Input [12]: [wr_item_sk#1, wr_refunded_cdemo_sk#2, wr_refunded_addr_sk#3, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#11, cd_education_status#12, cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] -(20) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] +(17) Scan parquet spark_catalog.default.reason +Output [2]: [r_reason_sk#16, r_reason_desc#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/reason] +PushedFilters: [IsNotNull(r_reason_sk)] +ReadSchema: struct -(21) ColumnarToRow [codegen id : 6] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] +(18) ColumnarToRow [codegen id : 3] +Input [2]: [r_reason_sk#16, r_reason_desc#17] -(22) Filter [codegen id : 6] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : (((isnotnull(cd_demo_sk#23) AND isnotnull(cd_marital_status#24)) AND isnotnull(cd_education_status#25)) AND ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Advanced Degree )) OR ((cd_marital_status#24 = S) AND (cd_education_status#25 = College ))) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = 2 yr Degree )))) +(19) Filter [codegen id : 3] +Input [2]: [r_reason_sk#16, r_reason_desc#17] +Condition : isnotnull(r_reason_sk#16) -(23) BroadcastExchange -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(20) BroadcastExchange +Input [2]: [r_reason_sk#16, r_reason_desc#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_refunded_cdemo_sk#11] -Right keys [1]: [cd_demo_sk#23] +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [wr_reason_sk#5] +Right keys [1]: [r_reason_sk#16] Join type: Inner -Join condition: ((((((cd_marital_status#24 = M) AND (cd_education_status#25 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#24 = S) AND (cd_education_status#25 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#24 = W) AND (cd_education_status#25 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) - -(25) Project [codegen id : 7] -Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#24, cd_education_status#25] -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#11, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] - -(26) Exchange -Input [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#24, cd_education_status#25] -Arguments: hashpartitioning(wr_returning_cdemo_sk#13, cd_marital_status#24, cd_education_status#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Join condition: None -(27) Sort [codegen id : 8] -Input [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#24, cd_education_status#25] -Arguments: [wr_returning_cdemo_sk#13 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST], false, 0 +(22) Project [codegen id : 5] +Output [8]: [wr_item_sk#1, wr_refunded_addr_sk#3, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_desc#17] +Input [10]: [wr_item_sk#1, wr_refunded_addr_sk#3, wr_reason_sk#5, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_sk#16, r_reason_desc#17] -(28) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] +(23) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [3]: [cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] +(24) ColumnarToRow [codegen id : 4] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] -(30) Filter [codegen id : 9] -Input [3]: [cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] -Condition : ((isnotnull(cd_demo_sk#26) AND isnotnull(cd_marital_status#27)) AND isnotnull(cd_education_status#28)) +(25) Filter [codegen id : 4] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Condition : (((isnotnull(ca_country#20) AND (ca_country#20 = United States)) AND isnotnull(ca_address_sk#18)) AND ((ca_state#19 IN (IN,OH,NJ) OR ca_state#19 IN (WI,CT,KY)) OR ca_state#19 IN (LA,IA,AR))) -(31) Exchange -Input [3]: [cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] -Arguments: hashpartitioning(cd_demo_sk#26, cd_marital_status#27, cd_education_status#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(26) Project [codegen id : 4] +Output [2]: [ca_address_sk#18, ca_state#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] -(32) Sort [codegen id : 10] -Input [3]: [cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST], false, 0 +(27) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(33) SortMergeJoin [codegen id : 14] -Left keys [3]: [wr_returning_cdemo_sk#13, cd_marital_status#24, cd_education_status#25] -Right keys [3]: [cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] +(28) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [wr_refunded_addr_sk#3] +Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 14] -Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17] -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#24, cd_education_status#25, cd_demo_sk#26, cd_marital_status#27, cd_education_status#28] +(29) Project [codegen id : 5] +Output [8]: [wr_item_sk#1, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_desc#17, ca_state#19] +Input [10]: [wr_item_sk#1, wr_refunded_addr_sk#3, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_desc#17, ca_address_sk#18, ca_state#19] + +(30) Exchange +Input [8]: [wr_item_sk#1, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_desc#17, ca_state#19] +Arguments: hashpartitioning(wr_item_sk#1, wr_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(35) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] +(31) Sort [codegen id : 6] +Input [8]: [wr_item_sk#1, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_desc#17, ca_state#19] +Arguments: [wr_item_sk#1 ASC NULLS FIRST, wr_order_number#6 ASC NULLS FIRST], false, 0 + +(32) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#21, ws_web_page_sk#22, ws_order_number#23, ws_quantity#24, ws_sales_price#25, ws_net_profit#26, ws_sold_date_sk#27] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] +ReadSchema: struct -(36) ColumnarToRow [codegen id : 11] -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] +(33) ColumnarToRow [codegen id : 7] +Input [7]: [ws_item_sk#21, ws_web_page_sk#22, ws_order_number#23, ws_quantity#24, ws_sales_price#25, ws_net_profit#26, ws_sold_date_sk#27] -(37) Filter [codegen id : 11] -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((ca_state#30 IN (IN,OH,NJ) OR ca_state#30 IN (WI,CT,KY)) OR ca_state#30 IN (LA,IA,AR))) +(34) Filter [codegen id : 7] +Input [7]: [ws_item_sk#21, ws_web_page_sk#22, ws_order_number#23, ws_quantity#24, ws_sales_price#25, ws_net_profit#26, ws_sold_date_sk#27] +Condition : ((((isnotnull(ws_item_sk#21) AND isnotnull(ws_order_number#23)) AND isnotnull(ws_web_page_sk#22)) AND ((((ws_sales_price#25 >= 100.00) AND (ws_sales_price#25 <= 150.00)) OR ((ws_sales_price#25 >= 50.00) AND (ws_sales_price#25 <= 100.00))) OR ((ws_sales_price#25 >= 150.00) AND (ws_sales_price#25 <= 200.00)))) AND ((((ws_net_profit#26 >= 100.00) AND (ws_net_profit#26 <= 200.00)) OR ((ws_net_profit#26 >= 150.00) AND (ws_net_profit#26 <= 300.00))) OR ((ws_net_profit#26 >= 50.00) AND (ws_net_profit#26 <= 250.00)))) -(38) Project [codegen id : 11] -Output [2]: [ca_address_sk#29, ca_state#30] -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] +(35) Exchange +Input [7]: [ws_item_sk#21, ws_web_page_sk#22, ws_order_number#23, ws_quantity#24, ws_sales_price#25, ws_net_profit#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_item_sk#21, ws_order_number#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) BroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(36) Sort [codegen id : 8] +Input [7]: [ws_item_sk#21, ws_web_page_sk#22, ws_order_number#23, ws_quantity#24, ws_sales_price#25, ws_net_profit#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21 ASC NULLS FIRST, ws_order_number#23 ASC NULLS FIRST], false, 0 -(40) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [wr_refunded_addr_sk#12] -Right keys [1]: [ca_address_sk#29] +(37) SortMergeJoin [codegen id : 11] +Left keys [2]: [wr_item_sk#1, wr_order_number#6] +Right keys [2]: [ws_item_sk#21, ws_order_number#23] Join type: Inner -Join condition: ((((ca_state#30 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#30 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#30 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) +Join condition: (((((((cd_marital_status#14 = M) AND (cd_education_status#15 = Advanced Degree )) AND (ws_sales_price#25 >= 100.00)) AND (ws_sales_price#25 <= 150.00)) OR ((((cd_marital_status#14 = S) AND (cd_education_status#15 = College )) AND (ws_sales_price#25 >= 50.00)) AND (ws_sales_price#25 <= 100.00))) OR ((((cd_marital_status#14 = W) AND (cd_education_status#15 = 2 yr Degree )) AND (ws_sales_price#25 >= 150.00)) AND (ws_sales_price#25 <= 200.00))) AND ((((ca_state#19 IN (IN,OH,NJ) AND (ws_net_profit#26 >= 100.00)) AND (ws_net_profit#26 <= 200.00)) OR ((ca_state#19 IN (WI,CT,KY) AND (ws_net_profit#26 >= 150.00)) AND (ws_net_profit#26 <= 300.00))) OR ((ca_state#19 IN (LA,IA,AR) AND (ws_net_profit#26 >= 50.00)) AND (ws_net_profit#26 <= 250.00)))) -(41) Project [codegen id : 14] -Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17] -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, ca_address_sk#29, ca_state#30] +(38) Project [codegen id : 11] +Output [6]: [wr_fee#7, wr_refunded_cash#8, r_reason_desc#17, ws_web_page_sk#22, ws_quantity#24, ws_sold_date_sk#27] +Input [15]: [wr_item_sk#1, wr_order_number#6, wr_fee#7, wr_refunded_cash#8, cd_marital_status#14, cd_education_status#15, r_reason_desc#17, ca_state#19, ws_item_sk#21, ws_web_page_sk#22, ws_order_number#23, ws_quantity#24, ws_sales_price#25, ws_net_profit#26, ws_sold_date_sk#27] -(42) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#32] +(39) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#29] -(43) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#32] +(40) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(44) Project [codegen id : 14] -Output [4]: [ws_quantity#4, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17] -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, d_date_sk#32] +(41) Project [codegen id : 11] +Output [5]: [wr_fee#7, wr_refunded_cash#8, r_reason_desc#17, ws_web_page_sk#22, ws_quantity#24] +Input [7]: [wr_fee#7, wr_refunded_cash#8, r_reason_desc#17, ws_web_page_sk#22, ws_quantity#24, ws_sold_date_sk#27, d_date_sk#29] -(45) Scan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#33, r_reason_desc#34] +(42) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#30] Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/web_page] +PushedFilters: [IsNotNull(wp_web_page_sk)] +ReadSchema: struct -(46) ColumnarToRow [codegen id : 13] -Input [2]: [r_reason_sk#33, r_reason_desc#34] +(43) ColumnarToRow [codegen id : 10] +Input [1]: [wp_web_page_sk#30] -(47) Filter [codegen id : 13] -Input [2]: [r_reason_sk#33, r_reason_desc#34] -Condition : isnotnull(r_reason_sk#33) +(44) Filter [codegen id : 10] +Input [1]: [wp_web_page_sk#30] +Condition : isnotnull(wp_web_page_sk#30) -(48) BroadcastExchange -Input [2]: [r_reason_sk#33, r_reason_desc#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(45) BroadcastExchange +Input [1]: [wp_web_page_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [wr_reason_sk#14] -Right keys [1]: [r_reason_sk#33] +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_web_page_sk#22] +Right keys [1]: [wp_web_page_sk#30] Join type: Inner Join condition: None -(50) Project [codegen id : 14] -Output [4]: [ws_quantity#4, wr_fee#16, wr_refunded_cash#17, r_reason_desc#34] -Input [6]: [ws_quantity#4, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, r_reason_sk#33, r_reason_desc#34] +(47) Project [codegen id : 11] +Output [4]: [ws_quantity#24, wr_fee#7, wr_refunded_cash#8, r_reason_desc#17] +Input [6]: [wr_fee#7, wr_refunded_cash#8, r_reason_desc#17, ws_web_page_sk#22, ws_quantity#24, wp_web_page_sk#30] -(51) HashAggregate [codegen id : 14] -Input [4]: [ws_quantity#4, wr_fee#16, wr_refunded_cash#17, r_reason_desc#34] -Keys [1]: [r_reason_desc#34] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#17)), partial_avg(UnscaledValue(wr_fee#16))] -Aggregate Attributes [6]: [sum#35, count#36, sum#37, count#38, sum#39, count#40] -Results [7]: [r_reason_desc#34, sum#41, count#42, sum#43, count#44, sum#45, count#46] +(48) HashAggregate [codegen id : 11] +Input [4]: [ws_quantity#24, wr_fee#7, wr_refunded_cash#8, r_reason_desc#17] +Keys [1]: [r_reason_desc#17] +Functions [3]: [partial_avg(ws_quantity#24), partial_avg(UnscaledValue(wr_refunded_cash#8)), partial_avg(UnscaledValue(wr_fee#7))] +Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] +Results [7]: [r_reason_desc#17, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(52) Exchange -Input [7]: [r_reason_desc#34, sum#41, count#42, sum#43, count#44, sum#45, count#46] -Arguments: hashpartitioning(r_reason_desc#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(49) Exchange +Input [7]: [r_reason_desc#17, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Arguments: hashpartitioning(r_reason_desc#17, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(53) HashAggregate [codegen id : 15] -Input [7]: [r_reason_desc#34, sum#41, count#42, sum#43, count#44, sum#45, count#46] -Keys [1]: [r_reason_desc#34] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#17)), avg(UnscaledValue(wr_fee#16))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#47, avg(UnscaledValue(wr_refunded_cash#17))#48, avg(UnscaledValue(wr_fee#16))#49] -Results [4]: [substr(r_reason_desc#34, 1, 20) AS substr(r_reason_desc, 1, 20)#50, avg(ws_quantity#4)#47 AS avg(ws_quantity)#51, cast((avg(UnscaledValue(wr_refunded_cash#17))#48 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#52, cast((avg(UnscaledValue(wr_fee#16))#49 / 100.0) as decimal(11,6)) AS avg(wr_fee)#53] +(50) HashAggregate [codegen id : 12] +Input [7]: [r_reason_desc#17, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Keys [1]: [r_reason_desc#17] +Functions [3]: [avg(ws_quantity#24), avg(UnscaledValue(wr_refunded_cash#8)), avg(UnscaledValue(wr_fee#7))] +Aggregate Attributes [3]: [avg(ws_quantity#24)#43, avg(UnscaledValue(wr_refunded_cash#8))#44, avg(UnscaledValue(wr_fee#7))#45] +Results [4]: [substr(r_reason_desc#17, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#24)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#8))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#7))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] -(54) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#50, avg(ws_quantity)#51, avg(wr_refunded_cash)#52, avg(wr_fee)#53] -Arguments: 100, [substr(r_reason_desc, 1, 20)#50 ASC NULLS FIRST, avg(ws_quantity)#51 ASC NULLS FIRST, avg(wr_refunded_cash)#52 ASC NULLS FIRST, avg(wr_fee)#53 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#50, avg(ws_quantity)#51, avg(wr_refunded_cash)#52, avg(wr_fee)#53] +(51) TakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] +Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet spark_catalog.default.date_dim (55) +Subquery:1 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#28 +BroadcastExchange (56) ++- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet spark_catalog.default.date_dim (52) -(55) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#54] +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#29, d_year#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#54] - -(57) Filter [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#54] -Condition : ((isnotnull(d_year#54) AND (d_year#54 = 2000)) AND isnotnull(d_date_sk#32)) - -(58) Project [codegen id : 1] -Output [1]: [d_date_sk#32] -Input [2]: [d_date_sk#32, d_year#54] - -(59) BroadcastExchange -Input [1]: [d_date_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 14 Hosting Expression = Subquery scalar-subquery#19, [id=#20] -ObjectHashAggregate (66) -+- Exchange (65) - +- ObjectHashAggregate (64) - +- * Project (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet spark_catalog.default.customer_demographics (60) - +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#29, d_year#50] -(60) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] -ReadSchema: struct +(54) Filter [codegen id : 1] +Input [2]: [d_date_sk#29, d_year#50] +Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#29)) -(61) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] - -(62) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : (((isnotnull(cd_demo_sk#23) AND isnotnull(cd_marital_status#24)) AND isnotnull(cd_education_status#25)) AND ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Advanced Degree )) OR ((cd_marital_status#24 = S) AND (cd_education_status#25 = College ))) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = 2 yr Degree )))) - -(63) Project [codegen id : 1] -Output [1]: [cd_demo_sk#23] -Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] - -(64) ObjectHashAggregate -Input [1]: [cd_demo_sk#23] -Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(cd_demo_sk#23, 42), 159981, 2239471, 0, 0)] -Aggregate Attributes [1]: [buf#55] -Results [1]: [buf#56] - -(65) Exchange -Input [1]: [buf#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] - -(66) ObjectHashAggregate -Input [1]: [buf#56] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(cd_demo_sk#23, 42), 159981, 2239471, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(cd_demo_sk#23, 42), 159981, 2239471, 0, 0)#57] -Results [1]: [bloom_filter_agg(xxhash64(cd_demo_sk#23, 42), 159981, 2239471, 0, 0)#57 AS bloomFilter#58] - -Subquery:3 Hosting operator id = 14 Hosting Expression = Subquery scalar-subquery#21, [id=#22] -ObjectHashAggregate (73) -+- Exchange (72) - +- ObjectHashAggregate (71) - +- * Project (70) - +- * Filter (69) - +- * ColumnarToRow (68) - +- Scan parquet spark_catalog.default.customer_address (67) - - -(67) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] -ReadSchema: struct +(55) Project [codegen id : 1] +Output [1]: [d_date_sk#29] +Input [2]: [d_date_sk#29, d_year#50] -(68) ColumnarToRow [codegen id : 1] -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] - -(69) Filter [codegen id : 1] -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((ca_state#30 IN (IN,OH,NJ) OR ca_state#30 IN (WI,CT,KY)) OR ca_state#30 IN (LA,IA,AR))) - -(70) Project [codegen id : 1] -Output [1]: [ca_address_sk#29] -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] - -(71) ObjectHashAggregate -Input [1]: [ca_address_sk#29] -Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#29, 42), 152837, 2153999, 0, 0)] -Aggregate Attributes [1]: [buf#59] -Results [1]: [buf#60] - -(72) Exchange -Input [1]: [buf#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] - -(73) ObjectHashAggregate -Input [1]: [buf#60] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#29, 42), 152837, 2153999, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#29, 42), 152837, 2153999, 0, 0)#61] -Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#29, 42), 152837, 2153999, 0, 0)#61 AS bloomFilter#62] +(56) BroadcastExchange +Input [1]: [d_date_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt index 46c14e8bd6773..bb30e7460ebc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt @@ -1,117 +1,88 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (15) + WholeStageCodegen (12) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] InputAdapter Exchange [r_reason_desc] #1 - WholeStageCodegen (14) + WholeStageCodegen (11) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [wr_fee,wr_refunded_cash,r_reason_desc,ws_web_page_sk,ws_quantity] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - SortMergeJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - WholeStageCodegen (8) - Sort [wr_returning_cdemo_sk,cd_marital_status,cd_education_status] - InputAdapter - Exchange [wr_returning_cdemo_sk,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk,ws_order_number] - InputAdapter - Exchange [ws_item_sk,ws_order_number] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [wp_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_page [wp_web_page_sk] + Project [wr_fee,wr_refunded_cash,r_reason_desc,ws_web_page_sk,ws_quantity,ws_sold_date_sk] + SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number,cd_marital_status,cd_education_status,ws_sales_price,ca_state,ws_net_profit] + InputAdapter + WholeStageCodegen (6) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #2 + WholeStageCodegen (5) + Project [wr_item_sk,wr_order_number,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,r_reason_desc,ca_state] + BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk] + Project [wr_item_sk,wr_refunded_addr_sk,wr_order_number,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,r_reason_desc] + BroadcastHashJoin [wr_reason_sk,r_reason_sk] + Project [wr_item_sk,wr_refunded_addr_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + BroadcastHashJoin [wr_refunded_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + Project [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + BroadcastHashJoin [wr_returning_cdemo_sk,cd_demo_sk] + Project [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cd_demo_sk,cd_marital_status,cd_education_status] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - WholeStageCodegen (5) - Sort [wr_item_sk,wr_order_number] - InputAdapter - Exchange [wr_item_sk,wr_order_number] #6 - WholeStageCodegen (4) - Project [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] - Subquery #2 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(cd_demo_sk, 42), 159981, 2239471, 0, 0),bloomFilter,buf] - Exchange #7 - ObjectHashAggregate [cd_demo_sk] [buf,buf] - WholeStageCodegen (1) - Project [cd_demo_sk] - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - Subquery #3 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 152837, 2153999, 0, 0),bloomFilter,buf] - Exchange #8 - ObjectHashAggregate [ca_address_sk] [buf,buf] - WholeStageCodegen (1) - Project [ca_address_sk] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [cd_demo_sk,cd_marital_status,cd_education_status] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Filter [cd_demo_sk,cd_marital_status,cd_education_status] + BroadcastExchange #5 + WholeStageCodegen (3) + Filter [r_reason_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - Exchange [cd_demo_sk,cd_marital_status,cd_education_status] #10 - WholeStageCodegen (9) - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + Scan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + Project [ca_address_sk,ca_state] + Filter [ca_country,ca_address_sk,ca_state] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (11) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + WholeStageCodegen (8) + Sort [ws_item_sk,ws_order_number] + InputAdapter + Exchange [ws_item_sk,ws_order_number] #7 + WholeStageCodegen (7) + Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #8 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #8 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (13) - Filter [r_reason_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [wp_web_page_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + Scan parquet spark_catalog.default.web_page [wp_web_page_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index af6632f4fb608..aa4aa3c684e27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -156,7 +156,7 @@ Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7 Output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -164,7 +164,7 @@ Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] (25) Filter [codegen id : 4] Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Condition : ((isnotnull(cd_demo_sk#22) AND isnotnull(cd_marital_status#23)) AND isnotnull(cd_education_status#24)) +Condition : (((isnotnull(cd_demo_sk#22) AND isnotnull(cd_marital_status#23)) AND isnotnull(cd_education_status#24)) AND ((((cd_marital_status#23 = M) AND (cd_education_status#24 = Advanced Degree )) OR ((cd_marital_status#23 = S) AND (cd_education_status#24 = College ))) OR ((cd_marital_status#23 = W) AND (cd_education_status#24 = 2 yr Degree )))) (26) BroadcastExchange Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 2b7d9bfeaa1f8..dc2f1572fa108 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * SortMergeJoin Inner (19) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * SortMergeJoin Inner (20) :- * Sort (13) : +- Exchange (12) : +- * Project (11) @@ -18,11 +18,12 @@ TakeOrderedAndProject (24) : +- * Filter (7) : +- * ColumnarToRow (6) : +- Scan parquet spark_catalog.default.reason (5) - +- * Sort (18) - +- Exchange (17) - +- * Project (16) - +- * ColumnarToRow (15) - +- Scan parquet spark_catalog.default.store_sales (14) + +- * Sort (19) + +- Exchange (18) + +- * Project (17) + +- * Filter (16) + +- * ColumnarToRow (15) + +- Scan parquet spark_catalog.default.store_sales (14) (1) Scan parquet spark_catalog.default.store_returns @@ -87,52 +88,57 @@ Arguments: [sr_item_sk#1 ASC NULLS FIRST, sr_ticket_number#3 ASC NULLS FIRST], f Output [6]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12, ss_sold_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] Input [6]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12, ss_sold_date_sk#13] -(16) Project [codegen id : 4] +(16) Filter [codegen id : 4] +Input [6]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12, ss_sold_date_sk#13] +Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_ticket_number#10)) + +(17) Project [codegen id : 4] Output [5]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12] Input [6]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12, ss_sold_date_sk#13] -(17) Exchange +(18) Exchange Input [5]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12] Arguments: hashpartitioning(ss_item_sk#8, ss_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) Sort [codegen id : 5] +(19) Sort [codegen id : 5] Input [5]: [ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST, ss_ticket_number#10 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 6] +(20) SortMergeJoin [codegen id : 6] Left keys [2]: [sr_item_sk#1, sr_ticket_number#3] Right keys [2]: [ss_item_sk#8, ss_ticket_number#10] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [2]: [ss_customer_sk#9, CASE WHEN isnotnull(sr_return_quantity#4) THEN (cast((ss_quantity#11 - sr_return_quantity#4) as decimal(10,0)) * ss_sales_price#12) ELSE (cast(ss_quantity#11 as decimal(10,0)) * ss_sales_price#12) END AS act_sales#14] Input [8]: [sr_item_sk#1, sr_ticket_number#3, sr_return_quantity#4, ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12] -(21) HashAggregate [codegen id : 6] +(22) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#9, act_sales#14] Keys [1]: [ss_customer_sk#9] Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#9, sum#17, isEmpty#18] -(22) Exchange +(23) Exchange Input [3]: [ss_customer_sk#9, sum#17, isEmpty#18] Arguments: hashpartitioning(ss_customer_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(23) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#9, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#9] Functions [1]: [sum(act_sales#14)] Aggregate Attributes [1]: [sum(act_sales#14)#19] Results [2]: [ss_customer_sk#9, sum(act_sales#14)#19 AS sumsales#20] -(24) TakeOrderedAndProject +(25) TakeOrderedAndProject Input [2]: [ss_customer_sk#9, sumsales#20] Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#9 ASC NULLS FIRST], [ss_customer_sk#9, sumsales#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt index 630d3d7fcd838..92a735d0f3062 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt @@ -35,6 +35,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #4 WholeStageCodegen (4) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + Filter [ss_item_sk,ss_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 172c03565451e..a6e8d395c0db5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,138 +1,144 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * SortMergeJoin Inner (12) - : :- * Sort (5) - : : +- Exchange (4) - : : +- * Project (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- * Sort (11) - : +- Exchange (10) - : +- * Project (9) - : +- * Filter (8) - : +- * ColumnarToRow (7) - : +- Scan parquet spark_catalog.default.store_returns (6) - +- BroadcastExchange (18) - +- * Project (17) - +- * Filter (16) - +- * ColumnarToRow (15) - +- Scan parquet spark_catalog.default.reason (14) +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (14) + : +- * SortMergeJoin Inner (13) + : :- * Sort (6) + : : +- Exchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet spark_catalog.default.store_sales (1) + : +- * Sort (12) + : +- Exchange (11) + : +- * Project (10) + : +- * Filter (9) + : +- * ColumnarToRow (8) + : +- Scan parquet spark_catalog.default.store_returns (7) + +- BroadcastExchange (19) + +- * Project (18) + +- * Filter (17) + +- * ColumnarToRow (16) + +- Scan parquet spark_catalog.default.reason (15) (1) Scan parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(3) Project [codegen id : 1] +(3) Filter [codegen id : 1] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] +Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#3)) + +(4) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(4) Exchange +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(5) Sort [codegen id : 2] +(6) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST], false, 0 -(6) Scan parquet spark_catalog.default.store_returns +(7) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] +(8) ColumnarToRow [codegen id : 3] Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -(8) Filter [codegen id : 3] +(9) Filter [codegen id : 3] Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) -(9) Project [codegen id : 3] +(10) Project [codegen id : 3] Output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -(10) Exchange +(11) Exchange Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 6] +(13) SortMergeJoin [codegen id : 6] Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] Join type: Inner Join condition: None -(13) Project [codegen id : 6] +(14) Project [codegen id : 6] Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -(14) Scan parquet spark_catalog.default.reason +(15) Scan parquet spark_catalog.default.reason Output [2]: [r_reason_sk#12, r_reason_desc#13] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 5] +(16) ColumnarToRow [codegen id : 5] Input [2]: [r_reason_sk#12, r_reason_desc#13] -(16) Filter [codegen id : 5] +(17) Filter [codegen id : 5] Input [2]: [r_reason_sk#12, r_reason_desc#13] Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) -(17) Project [codegen id : 5] +(18) Project [codegen id : 5] Output [1]: [r_reason_sk#12] Input [2]: [r_reason_sk#12, r_reason_desc#13] -(18) BroadcastExchange +(19) BroadcastExchange Input [1]: [r_reason_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#8] Right keys [1]: [r_reason_sk#12] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -(21) HashAggregate [codegen id : 6] +(22) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(22) Exchange +(23) Exchange Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(23) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] Aggregate Attributes [1]: [sum(act_sales#14)#19] Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(24) TakeOrderedAndProject +(25) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#20] Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 350956593ae88..ad84f69a8836f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -16,9 +16,10 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #2 WholeStageCodegen (1) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + Filter [ss_item_sk,ss_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index fafd7fd75cbd7..2734ed4f2c9e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -202,7 +202,7 @@ Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20] Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (25) ColumnarToRow [codegen id : 5] @@ -210,7 +210,7 @@ Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] (26) Filter [codegen id : 5] Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] -Condition : isnotnull(i_item_sk#21) +Condition : (((isnotnull(i_item_sk#21) AND isnotnull(i_brand_id#22)) AND isnotnull(i_class_id#23)) AND isnotnull(i_category_id#24)) (27) BroadcastExchange Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 4db035858b224..60178a502526d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -144,7 +144,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #11 WholeStageCodegen (5) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 4d69899b3b17a..9d20da052ebbe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -162,7 +162,7 @@ Condition : isnotnull(cs_item_sk#17) Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] @@ -170,7 +170,7 @@ Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Condition : (((isnotnull(i_item_sk#19) AND isnotnull(i_brand_id#20)) AND isnotnull(i_class_id#21)) AND isnotnull(i_category_id#22)) (19) BroadcastExchange Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index b103e79d11894..6d70272249736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -125,7 +125,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index d4a848eceacba..36828848476da 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -243,7 +243,7 @@ Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20] Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (25) ColumnarToRow [codegen id : 5] @@ -251,7 +251,7 @@ Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] (26) Filter [codegen id : 5] Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] -Condition : isnotnull(i_item_sk#21) +Condition : (((isnotnull(i_item_sk#21) AND isnotnull(i_brand_id#22)) AND isnotnull(i_class_id#23)) AND isnotnull(i_category_id#24)) (27) BroadcastExchange Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index 84c70b046c7d8..f5f572eb0e8bb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -158,7 +158,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #13 WholeStageCodegen (5) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index cdad294331fea..2f1de5e4f6321 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -203,7 +203,7 @@ Condition : isnotnull(cs_item_sk#17) Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] @@ -211,7 +211,7 @@ Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Condition : (((isnotnull(i_item_sk#19) AND isnotnull(i_brand_id#20)) AND isnotnull(i_class_id#21)) AND isnotnull(i_category_id#22)) (19) BroadcastExchange Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index ed3b4abc008f0..7c5a533695288 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -139,7 +139,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #11 WholeStageCodegen (1) - Filter [i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 1d1b56a42ad55..2087f556d64fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (90) -+- * HashAggregate (89) - +- Exchange (88) - +- * HashAggregate (87) - +- Union (86) - :- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- Union (72) +TakeOrderedAndProject (91) ++- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- Union (87) + :- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- Union (73) : :- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) @@ -50,45 +50,46 @@ TakeOrderedAndProject (90) : : : +- * ColumnarToRow (32) : : : +- Scan parquet spark_catalog.default.catalog_page (31) : : +- ReusedExchange (37) - : +- * HashAggregate (71) - : +- Exchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- Union (59) + : +- * HashAggregate (72) + : +- Exchange (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- * Project (66) + : : +- * BroadcastHashJoin Inner BuildRight (65) + : : :- Union (60) : : : :- * Project (46) : : : : +- * Filter (45) : : : : +- * ColumnarToRow (44) : : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : : +- * Project (58) - : : : +- * SortMergeJoin Inner (57) - : : : :- * Sort (50) - : : : : +- Exchange (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : : +- * Sort (56) - : : : +- Exchange (55) - : : : +- * Project (54) - : : : +- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet spark_catalog.default.web_sales (51) - : : +- BroadcastExchange (63) - : : +- * Filter (62) - : : +- * ColumnarToRow (61) - : : +- Scan parquet spark_catalog.default.web_site (60) - : +- ReusedExchange (66) - :- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * HashAggregate (77) - : +- ReusedExchange (76) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- ReusedExchange (81) + : : : +- * Project (59) + : : : +- * SortMergeJoin Inner (58) + : : : :- * Sort (51) + : : : : +- Exchange (50) + : : : : +- * Filter (49) + : : : : +- * ColumnarToRow (48) + : : : : +- Scan parquet spark_catalog.default.web_returns (47) + : : : +- * Sort (57) + : : : +- Exchange (56) + : : : +- * Project (55) + : : : +- * Filter (54) + : : : +- * ColumnarToRow (53) + : : : +- Scan parquet spark_catalog.default.web_sales (52) + : : +- BroadcastExchange (64) + : : +- * Filter (63) + : : +- * ColumnarToRow (62) + : : +- Scan parquet spark_catalog.default.web_site (61) + : +- ReusedExchange (67) + :- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * HashAggregate (78) + : +- ReusedExchange (77) + +- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- * HashAggregate (83) + +- ReusedExchange (82) (1) Scan parquet spark_catalog.default.store_sales @@ -159,7 +160,7 @@ Join condition: None Output [6]: [date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#23] Input [8]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#22, s_store_id#23] -(16) ReusedExchange [Reuses operator id: 95] +(16) ReusedExchange [Reuses operator id: 96] Output [1]: [d_date_sk#24] (17) BroadcastHashJoin [codegen id : 5] @@ -258,7 +259,7 @@ Join condition: None Output [6]: [date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#63] Input [8]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_sk#62, cp_catalog_page_id#63] -(37) ReusedExchange [Reuses operator id: 95] +(37) ReusedExchange [Reuses operator id: 96] Output [1]: [d_date_sk#64] (38) BroadcastHashJoin [codegen id : 11] @@ -313,245 +314,250 @@ Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (48) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -(49) Exchange +(49) Filter [codegen id : 14] +Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +Condition : (isnotnull(wr_item_sk#92) AND isnotnull(wr_order_number#93)) + +(50) Exchange Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] Arguments: hashpartitioning(wr_item_sk#92, wr_order_number#93, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) Sort [codegen id : 15] +(51) Sort [codegen id : 15] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] Arguments: [wr_item_sk#92 ASC NULLS FIRST, wr_order_number#93 ASC NULLS FIRST], false, 0 -(51) Scan parquet spark_catalog.default.web_sales +(52) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 16] +(53) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(53) Filter [codegen id : 16] +(54) Filter [codegen id : 16] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) -(54) Project [codegen id : 16] +(55) Project [codegen id : 16] Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(55) Exchange +(56) Exchange Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Arguments: hashpartitioning(ws_item_sk#97, ws_order_number#99, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(56) Sort [codegen id : 17] +(57) Sort [codegen id : 17] Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Arguments: [ws_item_sk#97 ASC NULLS FIRST, ws_order_number#99 ASC NULLS FIRST], false, 0 -(57) SortMergeJoin [codegen id : 18] +(58) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#92, wr_order_number#93] Right keys [2]: [ws_item_sk#97, ws_order_number#99] Join type: Inner Join condition: None -(58) Project [codegen id : 18] +(59) Project [codegen id : 18] Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -(59) Union +(60) Union -(60) Scan parquet spark_catalog.default.web_site +(61) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#107, web_site_id#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 19] +(62) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] -(62) Filter [codegen id : 19] +(63) Filter [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] Condition : isnotnull(web_site_sk#107) -(63) BroadcastExchange +(64) BroadcastExchange Input [2]: [web_site_sk#107, web_site_id#108] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(64) BroadcastHashJoin [codegen id : 21] +(65) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#86] Right keys [1]: [web_site_sk#107] Join type: Inner Join condition: None -(65) Project [codegen id : 21] +(66) Project [codegen id : 21] Output [6]: [date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108] Input [8]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#107, web_site_id#108] -(66) ReusedExchange [Reuses operator id: 95] +(67) ReusedExchange [Reuses operator id: 96] Output [1]: [d_date_sk#109] -(67) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(68) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108] Input [7]: [date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108, d_date_sk#109] -(69) HashAggregate [codegen id : 21] +(70) HashAggregate [codegen id : 21] Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#108] Keys [1]: [web_site_id#108] Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] Results [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117] -(70) Exchange +(71) Exchange Input [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117] Arguments: hashpartitioning(web_site_id#108, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(71) HashAggregate [codegen id : 22] +(72) HashAggregate [codegen id : 22] Input [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117] Keys [1]: [web_site_id#108] Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#108) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#126] -(72) Union +(73) Union -(73) HashAggregate [codegen id : 23] +(74) HashAggregate [codegen id : 23] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] Results [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(74) Exchange +(75) Exchange Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(75) HashAggregate [codegen id : 24] +(76) HashAggregate [codegen id : 24] Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] Results [5]: [channel#37, id#38, cast(sum(sales#39)#139 as decimal(37,2)) AS sales#142, cast(sum(returns#40)#140 as decimal(37,2)) AS returns#143, cast(sum(profit#41)#141 as decimal(38,2)) AS profit#144] -(76) ReusedExchange [Reuses operator id: 74] +(77) ReusedExchange [Reuses operator id: 75] Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(77) HashAggregate [codegen id : 48] +(78) HashAggregate [codegen id : 48] Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] Keys [2]: [channel#145, id#146] Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] Aggregate Attributes [3]: [sum(sales#153)#139, sum(returns#154)#140, sum(profit#155)#141] Results [4]: [channel#145, sum(sales#153)#139 AS sales#156, sum(returns#154)#140 AS returns#157, sum(profit#155)#141 AS profit#158] -(78) HashAggregate [codegen id : 48] +(79) HashAggregate [codegen id : 48] Input [4]: [channel#145, sales#156, returns#157, profit#158] Keys [1]: [channel#145] Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] Aggregate Attributes [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] Results [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -(79) Exchange +(80) Exchange Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Arguments: hashpartitioning(channel#145, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(80) HashAggregate [codegen id : 49] +(81) HashAggregate [codegen id : 49] Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Keys [1]: [channel#145] Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] Aggregate Attributes [3]: [sum(sales#156)#171, sum(returns#157)#172, sum(profit#158)#173] Results [5]: [channel#145, null AS id#174, sum(sales#156)#171 AS sum(sales)#175, sum(returns#157)#172 AS sum(returns)#176, sum(profit#158)#173 AS sum(profit)#177] -(81) ReusedExchange [Reuses operator id: 74] +(82) ReusedExchange [Reuses operator id: 75] Output [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] -(82) HashAggregate [codegen id : 73] +(83) HashAggregate [codegen id : 73] Input [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] Keys [2]: [channel#178, id#179] Functions [3]: [sum(sales#186), sum(returns#187), sum(profit#188)] Aggregate Attributes [3]: [sum(sales#186)#139, sum(returns#187)#140, sum(profit#188)#141] Results [3]: [sum(sales#186)#139 AS sales#189, sum(returns#187)#140 AS returns#190, sum(profit#188)#141 AS profit#191] -(83) HashAggregate [codegen id : 73] +(84) HashAggregate [codegen id : 73] Input [3]: [sales#189, returns#190, profit#191] Keys: [] Functions [3]: [partial_sum(sales#189), partial_sum(returns#190), partial_sum(profit#191)] Aggregate Attributes [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] Results [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] -(84) Exchange +(85) Exchange Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(85) HashAggregate [codegen id : 74] +(86) HashAggregate [codegen id : 74] Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] Keys: [] Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] Aggregate Attributes [3]: [sum(sales#189)#204, sum(returns#190)#205, sum(profit#191)#206] Results [5]: [null AS channel#207, null AS id#208, sum(sales#189)#204 AS sum(sales)#209, sum(returns#190)#205 AS sum(returns)#210, sum(profit#191)#206 AS sum(profit)#211] -(86) Union +(87) Union -(87) HashAggregate [codegen id : 75] +(88) HashAggregate [codegen id : 75] Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -(88) Exchange +(89) Exchange Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Arguments: hashpartitioning(channel#37, id#38, sales#142, returns#143, profit#144, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(89) HashAggregate [codegen id : 76] +(90) HashAggregate [codegen id : 76] Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -(90) TakeOrderedAndProject +(91) TakeOrderedAndProject Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#142, returns#143, profit#144] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (95) -+- * Project (94) - +- * Filter (93) - +- * ColumnarToRow (92) - +- Scan parquet spark_catalog.default.date_dim (91) +BroadcastExchange (96) ++- * Project (95) + +- * Filter (94) + +- * ColumnarToRow (93) + +- Scan parquet spark_catalog.default.date_dim (92) -(91) Scan parquet spark_catalog.default.date_dim +(92) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#212] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 1] +(93) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_date#212] -(93) Filter [codegen id : 1] +(94) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_date#212] Condition : (((isnotnull(d_date#212) AND (d_date#212 >= 1998-08-04)) AND (d_date#212 <= 1998-08-18)) AND isnotnull(d_date_sk#24)) -(94) Project [codegen id : 1] +(95) Project [codegen id : 1] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#212] -(95) BroadcastExchange +(96) BroadcastExchange Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index f8cc4833beed6..33a7452ff905d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -120,10 +120,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #9 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 34c6ecf3cf2fa..8902c022c1342 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- * HashAggregate (86) - +- Exchange (85) - +- * HashAggregate (84) - +- Union (83) - :- * HashAggregate (72) - : +- Exchange (71) - : +- * HashAggregate (70) - : +- Union (69) +TakeOrderedAndProject (88) ++- * HashAggregate (87) + +- Exchange (86) + +- * HashAggregate (85) + +- Union (84) + :- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- Union (70) : :- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) @@ -50,42 +50,43 @@ TakeOrderedAndProject (87) : : +- * Filter (36) : : +- * ColumnarToRow (35) : : +- Scan parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- Union (56) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (60) + : : +- * BroadcastHashJoin Inner BuildRight (59) + : : :- Union (57) : : : :- * Project (46) : : : : +- * Filter (45) : : : : +- * ColumnarToRow (44) : : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : : :- BroadcastExchange (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : : +- * Project (53) - : : : +- * Filter (52) - : : : +- * ColumnarToRow (51) - : : : +- Scan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * Filter (62) - : +- * ColumnarToRow (61) - : +- Scan parquet spark_catalog.default.web_site (60) - :- * HashAggregate (77) - : +- Exchange (76) - : +- * HashAggregate (75) - : +- * HashAggregate (74) - : +- ReusedExchange (73) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * HashAggregate (79) - +- ReusedExchange (78) + : : : +- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildLeft (55) + : : : :- BroadcastExchange (50) + : : : : +- * Filter (49) + : : : : +- * ColumnarToRow (48) + : : : : +- Scan parquet spark_catalog.default.web_returns (47) + : : : +- * Project (54) + : : : +- * Filter (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet spark_catalog.default.web_sales (51) + : : +- ReusedExchange (58) + : +- BroadcastExchange (64) + : +- * Filter (63) + : +- * ColumnarToRow (62) + : +- Scan parquet spark_catalog.default.web_site (61) + :- * HashAggregate (78) + : +- Exchange (77) + : +- * HashAggregate (76) + : +- * HashAggregate (75) + : +- ReusedExchange (74) + +- * HashAggregate (83) + +- Exchange (82) + +- * HashAggregate (81) + +- * HashAggregate (80) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -128,7 +129,7 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (9) Union -(10) ReusedExchange [Reuses operator id: 92] +(10) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#22] (11) BroadcastHashJoin [codegen id : 5] @@ -227,7 +228,7 @@ Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_retur (30) Union -(31) ReusedExchange [Reuses operator id: 92] +(31) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#62] (32) BroadcastHashJoin [codegen id : 11] @@ -310,233 +311,238 @@ Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (48) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -(49) BroadcastExchange +(49) Filter [codegen id : 14] Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Condition : (isnotnull(wr_item_sk#92) AND isnotnull(wr_order_number#93)) -(50) Scan parquet spark_catalog.default.web_sales +(50) BroadcastExchange +Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=5] + +(51) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(51) ColumnarToRow +(52) ColumnarToRow Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(52) Filter +(53) Filter Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) -(53) Project +(54) Project Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(54) BroadcastHashJoin [codegen id : 15] +(55) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#92, wr_order_number#93] Right keys [2]: [ws_item_sk#97, ws_order_number#99] Join type: Inner Join condition: None -(55) Project [codegen id : 15] +(56) Project [codegen id : 15] Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -(56) Union +(57) Union -(57) ReusedExchange [Reuses operator id: 92] +(58) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#107] -(58) BroadcastHashJoin [codegen id : 18] +(59) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(59) Project [codegen id : 18] +(60) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91] Input [7]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, d_date_sk#107] -(60) Scan parquet spark_catalog.default.web_site +(61) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#108, web_site_id#109] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 17] +(62) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] -(62) Filter [codegen id : 17] +(63) Filter [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] Condition : isnotnull(web_site_sk#108) -(63) BroadcastExchange +(64) BroadcastExchange Input [2]: [web_site_sk#108, web_site_id#109] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(64) BroadcastHashJoin [codegen id : 18] +(65) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#86] Right keys [1]: [web_site_sk#108] Join type: Inner Join condition: None -(65) Project [codegen id : 18] +(66) Project [codegen id : 18] Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] Input [7]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#108, web_site_id#109] -(66) HashAggregate [codegen id : 18] +(67) HashAggregate [codegen id : 18] Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] Keys [1]: [web_site_id#109] Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] Results [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -(67) Exchange +(68) Exchange Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] Arguments: hashpartitioning(web_site_id#109, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(68) HashAggregate [codegen id : 19] +(69) HashAggregate [codegen id : 19] Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] Keys [1]: [web_site_id#109] Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#109) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#126] -(69) Union +(70) Union -(70) HashAggregate [codegen id : 20] +(71) HashAggregate [codegen id : 20] Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] Keys [2]: [channel#37, id#38] Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] Aggregate Attributes [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] Results [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(71) Exchange +(72) Exchange Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(72) HashAggregate [codegen id : 21] +(73) HashAggregate [codegen id : 21] Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Keys [2]: [channel#37, id#38] Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] Results [5]: [channel#37, id#38, cast(sum(sales#39)#139 as decimal(37,2)) AS sales#142, cast(sum(returns#40)#140 as decimal(37,2)) AS returns#143, cast(sum(profit#41)#141 as decimal(38,2)) AS profit#144] -(73) ReusedExchange [Reuses operator id: 71] +(74) ReusedExchange [Reuses operator id: 72] Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(74) HashAggregate [codegen id : 42] +(75) HashAggregate [codegen id : 42] Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] Keys [2]: [channel#145, id#146] Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] Aggregate Attributes [3]: [sum(sales#153)#139, sum(returns#154)#140, sum(profit#155)#141] Results [4]: [channel#145, sum(sales#153)#139 AS sales#156, sum(returns#154)#140 AS returns#157, sum(profit#155)#141 AS profit#158] -(75) HashAggregate [codegen id : 42] +(76) HashAggregate [codegen id : 42] Input [4]: [channel#145, sales#156, returns#157, profit#158] Keys [1]: [channel#145] Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] Aggregate Attributes [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] Results [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -(76) Exchange +(77) Exchange Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Arguments: hashpartitioning(channel#145, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(77) HashAggregate [codegen id : 43] +(78) HashAggregate [codegen id : 43] Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Keys [1]: [channel#145] Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] Aggregate Attributes [3]: [sum(sales#156)#171, sum(returns#157)#172, sum(profit#158)#173] Results [5]: [channel#145, null AS id#174, sum(sales#156)#171 AS sum(sales)#175, sum(returns#157)#172 AS sum(returns)#176, sum(profit#158)#173 AS sum(profit)#177] -(78) ReusedExchange [Reuses operator id: 71] +(79) ReusedExchange [Reuses operator id: 72] Output [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] -(79) HashAggregate [codegen id : 64] +(80) HashAggregate [codegen id : 64] Input [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] Keys [2]: [channel#178, id#179] Functions [3]: [sum(sales#186), sum(returns#187), sum(profit#188)] Aggregate Attributes [3]: [sum(sales#186)#139, sum(returns#187)#140, sum(profit#188)#141] Results [3]: [sum(sales#186)#139 AS sales#189, sum(returns#187)#140 AS returns#190, sum(profit#188)#141 AS profit#191] -(80) HashAggregate [codegen id : 64] +(81) HashAggregate [codegen id : 64] Input [3]: [sales#189, returns#190, profit#191] Keys: [] Functions [3]: [partial_sum(sales#189), partial_sum(returns#190), partial_sum(profit#191)] Aggregate Attributes [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] Results [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] -(81) Exchange +(82) Exchange Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(82) HashAggregate [codegen id : 65] +(83) HashAggregate [codegen id : 65] Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] Keys: [] Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] Aggregate Attributes [3]: [sum(sales#189)#204, sum(returns#190)#205, sum(profit#191)#206] Results [5]: [null AS channel#207, null AS id#208, sum(sales#189)#204 AS sum(sales)#209, sum(returns#190)#205 AS sum(returns)#210, sum(profit#191)#206 AS sum(profit)#211] -(83) Union +(84) Union -(84) HashAggregate [codegen id : 66] +(85) HashAggregate [codegen id : 66] Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -(85) Exchange +(86) Exchange Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Arguments: hashpartitioning(channel#37, id#38, sales#142, returns#143, profit#144, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(86) HashAggregate [codegen id : 67] +(87) HashAggregate [codegen id : 67] Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Functions: [] Aggregate Attributes: [] Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -(87) TakeOrderedAndProject +(88) TakeOrderedAndProject Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#142, returns#143, profit#144] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (92) -+- * Project (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (93) ++- * Project (92) + +- * Filter (91) + +- * ColumnarToRow (90) + +- Scan parquet spark_catalog.default.date_dim (89) -(88) Scan parquet spark_catalog.default.date_dim +(89) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#212] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 1] +(90) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_date#212] -(90) Filter [codegen id : 1] +(91) Filter [codegen id : 1] Input [2]: [d_date_sk#22, d_date#212] Condition : (((isnotnull(d_date#212) AND (d_date#212 >= 1998-08-04)) AND (d_date#212 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(91) Project [codegen id : 1] +(92) Project [codegen id : 1] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#212] -(92) BroadcastExchange +(93) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index c7d72bb6c5bd7..7c8f9566db242 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -117,10 +117,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [wr_item_sk,wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow