From eb0096ace914f48ce173771b2e2a4c42dc854dd8 Mon Sep 17 00:00:00 2001 From: Kapil Kumar Singh Date: Fri, 3 Mar 2023 12:08:54 +0530 Subject: [PATCH 1/6] Run infer filter after RewritePredicateSubquery --- .../sql/catalyst/optimizer/Optimizer.scala | 4 +- .../approved-plans-v1_4/q10.sf100/explain.txt | 242 +++++---- .../q10.sf100/simplified.txt | 25 +- .../approved-plans-v1_4/q10/explain.txt | 190 +++---- .../approved-plans-v1_4/q10/simplified.txt | 25 +- .../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/q16.sf100/explain.txt | 252 ++++----- .../q16.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q16/explain.txt | 168 +++--- .../approved-plans-v1_4/q16/simplified.txt | 16 +- .../q23a.sf100/explain.txt | 510 +++++++++--------- .../q23a.sf100/simplified.txt | 32 +- .../approved-plans-v1_4/q23a/explain.txt | 406 +++++++------- .../approved-plans-v1_4/q23a/simplified.txt | 32 +- .../q23b.sf100/explain.txt | 8 +- .../q23b.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q23b/explain.txt | 8 +- .../approved-plans-v1_4/q23b/simplified.txt | 4 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 8 +- .../q33.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 8 +- .../approved-plans-v1_4/q33/simplified.txt | 4 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 222 ++++---- .../q35.sf100/simplified.txt | 25 +- .../approved-plans-v1_4/q35/explain.txt | 186 +++---- .../approved-plans-v1_4/q35/simplified.txt | 25 +- .../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/q56.sf100/explain.txt | 8 +- .../q56.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q56/explain.txt | 8 +- .../approved-plans-v1_4/q56/simplified.txt | 4 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 8 +- .../q58.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q58/explain.txt | 8 +- .../approved-plans-v1_4/q58/simplified.txt | 4 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 8 +- .../q60.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q60/explain.txt | 8 +- .../approved-plans-v1_4/q60/simplified.txt | 4 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 238 ++++---- .../q69.sf100/simplified.txt | 43 +- .../approved-plans-v1_4/q69/explain.txt | 198 +++---- .../approved-plans-v1_4/q69/simplified.txt | 43 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 147 ++--- .../q70.sf100/simplified.txt | 67 +-- .../approved-plans-v1_4/q70/explain.txt | 147 ++--- .../approved-plans-v1_4/q70/simplified.txt | 65 +-- .../approved-plans-v1_4/q8.sf100/explain.txt | 3 +- .../approved-plans-v1_4/q8/explain.txt | 3 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 67 +-- .../q83.sf100/simplified.txt | 11 +- .../approved-plans-v1_4/q83/explain.txt | 67 +-- .../approved-plans-v1_4/q83/simplified.txt | 11 +- .../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-v1_4/q94.sf100/explain.txt | 252 ++++----- .../q94.sf100/simplified.txt | 16 +- .../approved-plans-v1_4/q94/explain.txt | 168 +++--- .../approved-plans-v1_4/q94/simplified.txt | 16 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 4 +- .../q95.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q95/explain.txt | 4 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../q10a.sf100/explain.txt | 230 ++++---- .../q10a.sf100/simplified.txt | 43 +- .../approved-plans-v2_7/q10a/explain.txt | 194 ++++--- .../approved-plans-v2_7/q10a/simplified.txt | 43 +- .../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/q35.sf100/explain.txt | 222 ++++---- .../q35.sf100/simplified.txt | 25 +- .../approved-plans-v2_7/q35/explain.txt | 186 +++---- .../approved-plans-v2_7/q35/simplified.txt | 25 +- .../q35a.sf100/explain.txt | 222 ++++---- .../q35a.sf100/simplified.txt | 43 +- .../approved-plans-v2_7/q35a/explain.txt | 190 ++++--- .../approved-plans-v2_7/q35a/simplified.txt | 43 +- .../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 +- .../q70a.sf100/explain.txt | 199 +++---- .../q70a.sf100/simplified.txt | 79 +-- .../approved-plans-v2_7/q70a/explain.txt | 199 +++---- .../approved-plans-v2_7/q70a/simplified.txt | 77 +-- .../tpch-plan-stability/q18/explain.txt | 98 ++-- .../tpch-plan-stability/q18/simplified.txt | 7 +- .../tpch-plan-stability/q19/explain.txt | 18 +- .../tpch-plan-stability/q20/explain.txt | 8 +- .../tpch-plan-stability/q20/simplified.txt | 4 +- .../tpch-plan-stability/q21/explain.txt | 132 ++--- .../tpch-plan-stability/q21/simplified.txt | 9 +- .../tpch-plan-stability/q22/explain.txt | 72 +-- .../tpch-plan-stability/q22/simplified.txt | 7 +- .../tpch-plan-stability/q4/explain.txt | 8 +- .../tpch-plan-stability/q4/simplified.txt | 4 +- 114 files changed, 3875 insertions(+), 3508 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ca55a28160509..cb63592608869 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 @@ -231,7 +231,9 @@ abstract class Optimizer(catalogManager: CatalogManager) ColumnPruning, CollapseProject, RemoveRedundantAliases, - RemoveNoopOperators) :+ + RemoveNoopOperators, + InferFiltersFromConstraints, + PushDownPredicates) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+ Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index d3434fd6be977..25bf7e3dde34a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -1,62 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * SortMergeJoin Inner (46) - :- * Sort (40) - : +- Exchange (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * Filter (30) - : : +- * SortMergeJoin ExistenceJoin(exists#1) (29) - : : :- * SortMergeJoin ExistenceJoin(exists#2) (21) - : : : :- * SortMergeJoin LeftSemi (13) +TakeOrderedAndProject (52) ++- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- * Project (48) + +- * SortMergeJoin Inner (47) + :- * Sort (41) + : +- Exchange (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (32) + : : +- * Filter (31) + : : +- * SortMergeJoin ExistenceJoin(exists#1) (30) + : : :- * SortMergeJoin ExistenceJoin(exists#2) (22) + : : : :- * SortMergeJoin LeftSemi (14) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (8) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * ColumnarToRow (15) - : : : : +- Scan parquet spark_catalog.default.web_sales (14) - : : : +- ReusedExchange (16) - : : +- * Sort (28) - : : +- Exchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet spark_catalog.default.customer_address (32) - +- * Sort (45) - +- Exchange (44) - +- * Filter (43) - +- * ColumnarToRow (42) - +- Scan parquet spark_catalog.default.customer_demographics (41) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet spark_catalog.default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet spark_catalog.default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet spark_catalog.default.customer_address (33) + +- * Sort (46) + +- Exchange (45) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet spark_catalog.default.customer_demographics (42) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -64,7 +65,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND might_contain(Subquery scalar-subquery#6, [id=#7], xxhash64(c_current_addr_sk#5, 42))) +Condition : (((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) AND might_contain(Subquery scalar-subquery#6, [id=#7], xxhash64(c_current_addr_sk#5, 42))) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -79,256 +80,261 @@ Output [2]: [ss_customer_sk#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#8, ss_sold_date_sk#9] -(8) ReusedExchange [Reuses operator id: 63] +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#8, ss_sold_date_sk#9] +Condition : isnotnull(ss_customer_sk#8) + +(9) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#11] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [1]: [ss_customer_sk#8] Input [3]: [ss_customer_sk#8, ss_sold_date_sk#9, d_date_sk#11] -(11) Exchange +(12) Exchange Input [1]: [ss_customer_sk#8] Arguments: hashpartitioning(ss_customer_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [1]: [ss_customer_sk#8] Arguments: [ss_customer_sk#8 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#8] Join type: LeftSemi Join condition: None -(14) Scan parquet spark_catalog.default.web_sales +(15) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#10)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 8] +(16) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -(16) ReusedExchange [Reuses operator id: 63] +(17) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#14] -(17) BroadcastHashJoin [codegen id : 8] +(18) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(19) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#12] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] -(19) Exchange +(20) Exchange Input [1]: [ws_bill_customer_sk#12] Arguments: hashpartitioning(ws_bill_customer_sk#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) Sort [codegen id : 9] +(21) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#12] Arguments: [ws_bill_customer_sk#12 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin [codegen id : 10] +(22) SortMergeJoin [codegen id : 10] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(22) Scan parquet spark_catalog.default.catalog_sales +(23) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 12] +(24) ColumnarToRow [codegen id : 12] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -(24) ReusedExchange [Reuses operator id: 63] +(25) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#17] -(25) BroadcastHashJoin [codegen id : 12] +(26) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 12] +(27) Project [codegen id : 12] Output [1]: [cs_ship_customer_sk#15] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#17] -(27) Exchange +(28) Exchange Input [1]: [cs_ship_customer_sk#15] Arguments: hashpartitioning(cs_ship_customer_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) Sort [codegen id : 13] +(29) Sort [codegen id : 13] Input [1]: [cs_ship_customer_sk#15] Arguments: [cs_ship_customer_sk#15 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 15] +(30) SortMergeJoin [codegen id : 15] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(30) Filter [codegen id : 15] +(31) Filter [codegen id : 15] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 15] +(32) Project [codegen id : 15] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet spark_catalog.default.customer_address +(33) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#18, ca_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 14] +(34) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#18, ca_county#19] -(34) Filter [codegen id : 14] +(35) Filter [codegen id : 14] Input [2]: [ca_address_sk#18, ca_county#19] Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) -(35) Project [codegen id : 14] +(36) Project [codegen id : 14] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_county#19] -(36) BroadcastExchange +(37) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 15] +(38) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(38) Project [codegen id : 15] +(39) Project [codegen id : 15] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] -(39) Exchange +(40) Exchange Input [1]: [c_current_cdemo_sk#4] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 16] +(41) Sort [codegen id : 16] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.customer_demographics +(42) Scan parquet spark_catalog.default.customer_demographics Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 17] +(43) ColumnarToRow [codegen id : 17] Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) Filter [codegen id : 17] +(44) Filter [codegen id : 17] Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#20) -(44) Exchange +(45) Exchange Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: hashpartitioning(cd_demo_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(45) Sort [codegen id : 18] +(46) Sort [codegen id : 18] Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#20 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin [codegen id : 19] +(47) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(47) Project [codegen id : 19] +(48) Project [codegen id : 19] Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(48) HashAggregate [codegen id : 19] +(49) HashAggregate [codegen id : 19] Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(49) Exchange +(50) Exchange Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(50) HashAggregate [codegen id : 20] +(51) HashAggregate [codegen id : 20] Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(51) TakeOrderedAndProject +(52) TakeOrderedAndProject Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#6, [id=#7] -ObjectHashAggregate (58) -+- Exchange (57) - +- ObjectHashAggregate (56) - +- * Project (55) - +- * Filter (54) - +- * ColumnarToRow (53) - +- Scan parquet spark_catalog.default.customer_address (52) +ObjectHashAggregate (59) ++- Exchange (58) + +- ObjectHashAggregate (57) + +- * Project (56) + +- * Filter (55) + +- * ColumnarToRow (54) + +- Scan parquet spark_catalog.default.customer_address (53) -(52) Scan parquet spark_catalog.default.customer_address +(53) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#18, ca_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [2]: [ca_address_sk#18, ca_county#19] -(54) Filter [codegen id : 1] +(55) Filter [codegen id : 1] Input [2]: [ca_address_sk#18, ca_county#19] Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) -(55) Project [codegen id : 1] +(56) Project [codegen id : 1] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_county#19] -(56) ObjectHashAggregate +(57) ObjectHashAggregate Input [1]: [ca_address_sk#18] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)] Aggregate Attributes [1]: [buf#38] Results [1]: [buf#39] -(57) Exchange +(58) Exchange Input [1]: [buf#39] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(58) ObjectHashAggregate +(59) ObjectHashAggregate Input [1]: [buf#39] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)] @@ -336,37 +342,37 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555 Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)#40 AS bloomFilter#41] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (63) -+- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet spark_catalog.default.date_dim (59) +BroadcastExchange (64) ++- * Project (63) + +- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet spark_catalog.default.date_dim (60) -(59) Scan parquet spark_catalog.default.date_dim +(60) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#42, d_moy#43] -(61) Filter [codegen id : 1] +(62) Filter [codegen id : 1] Input [3]: [d_date_sk#11, d_year#42, d_moy#43] Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 1)) AND (d_moy#43 <= 4)) AND isnotnull(d_date_sk#11)) -(62) Project [codegen id : 1] +(63) Project [codegen id : 1] Output [1]: [d_date_sk#11] Input [3]: [d_date_sk#11, d_year#42, d_moy#43] -(63) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 14 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index 9528756e264b5..50e9619e3430e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -30,7 +30,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 2555, 57765, 0, 0),bloomFilter,buf] Exchange #4 @@ -52,17 +52,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 2663b8a060153..123f5c7ae3eed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,54 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * Filter (26) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (25) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (18) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.customer_demographics (34) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * ColumnarToRow (13) + : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- Scan parquet spark_catalog.default.catalog_sales (19) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -56,231 +57,236 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 48] +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(7) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(9) BroadcastExchange +(10) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(11) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(11) Scan parquet spark_catalog.default.web_sales +(12) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 48] +(14) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(16) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(16) BroadcastExchange +(17) BroadcastExchange Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(18) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales +(19) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(20) ReusedExchange [Reuses operator id: 48] +(21) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] -(23) BroadcastExchange +(24) BroadcastExchange Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(26) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address +(28) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_county#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 7] +(29) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#16, ca_county#17] -(29) Filter [codegen id : 7] +(30) Filter [codegen id : 7] Input [2]: [ca_address_sk#16, ca_county#17] Condition : (ca_county#17 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#16)) -(30) Project [codegen id : 7] +(31) Project [codegen id : 7] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_county#17] -(31) BroadcastExchange +(32) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(34) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16] -(34) Scan parquet spark_catalog.default.customer_demographics +(35) Scan parquet spark_catalog.default.customer_demographics Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 8] +(36) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(36) Filter [codegen id : 8] +(37) Filter [codegen id : 8] Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Condition : isnotnull(cd_demo_sk#18) -(37) BroadcastExchange +(38) BroadcastExchange Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(40) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#27] Results [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] -(41) Exchange +(42) Exchange Input [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] Keys [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#29] Results [14]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#29 AS cnt1#30, cd_purchase_estimate#22, count(1)#29 AS cnt2#31, cd_credit_rating#23, count(1)#29 AS cnt3#32, cd_dep_count#24, count(1)#29 AS cnt4#33, cd_dep_employed_count#25, count(1)#29 AS cnt5#34, cd_dep_college_count#26, count(1)#29 AS cnt6#35] -(43) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [14]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#30, cd_purchase_estimate#22, cnt2#31, cd_credit_rating#23, cnt3#32, cd_dep_count#24, cnt4#33, cd_dep_employed_count#25, cnt5#34, cd_dep_college_count#26, cnt6#35] Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#30, cd_purchase_estimate#22, cnt2#31, cd_credit_rating#23, cnt3#32, cd_dep_count#24, cnt4#33, cd_dep_employed_count#25, cnt5#34, cd_dep_college_count#26, cnt6#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (48) -+- * Project (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (49) ++- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet spark_catalog.default.date_dim (45) -(44) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#36, d_moy#37] -(46) Filter [codegen id : 1] +(47) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#36, d_moy#37] Condition : (((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2002)) AND (d_moy#37 >= 1)) AND (d_moy#37 <= 4)) AND isnotnull(d_date_sk#9)) -(47) Project [codegen id : 1] +(48) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#36, d_moy#37] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 26499a07dd7ed..33157630a88d2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,17 +23,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter 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 e350459c60cea..f069135dca882 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 @@ -131,7 +131,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 17] @@ -139,7 +139,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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..6e37e751722db 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 @@ -74,7 +74,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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 50a1d636ffc4a..100a723e381c2 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 @@ -125,7 +125,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -133,7 +133,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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..bdcfb39025bac 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 @@ -74,7 +74,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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 99573a6202873..feab76d741807 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 @@ -110,7 +110,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 17] @@ -118,7 +118,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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 edd3486498691..0d6f76d6e73bb 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 @@ -72,7 +72,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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 bc2fc87cb1430..6d5c59fbbb128 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 @@ -104,7 +104,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -112,7 +112,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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 8d8dcccd5d70c..fb2e53c91965a 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 @@ -72,7 +72,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index aadf72d0af219..4249b82d0da97 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -1,56 +1,58 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * SortMergeJoin LeftAnti (21) + : : : :- * Project (14) + : : : : +- * SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet spark_catalog.default.customer_address (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.call_center (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.date_dim (34) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet spark_catalog.default.catalog_returns (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet spark_catalog.default.customer_address (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet spark_catalog.default.call_center (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet spark_catalog.default.date_dim (36) (1) Scan parquet spark_catalog.default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -58,7 +60,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : (((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(cs_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(cs_call_center_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(cs_ship_date_sk#1, 42))) +Condition : (((((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(cs_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(cs_call_center_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(cs_ship_date_sk#1, 42))) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -76,182 +78,192 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#15, cs_order_number#16, cs_sold_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#15, cs_order_number#16, cs_sold_date_sk#17] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [cs_warehouse_sk#15, cs_order_number#16, cs_sold_date_sk#17] +Condition : (isnotnull(cs_order_number#16) AND isnotnull(cs_warehouse_sk#15)) + +(10) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#15, cs_order_number#16] Input [3]: [cs_warehouse_sk#15, cs_order_number#16, cs_sold_date_sk#17] -(10) Exchange +(11) Exchange Input [2]: [cs_warehouse_sk#15, cs_order_number#16] Arguments: hashpartitioning(cs_order_number#16, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#15, cs_order_number#16] Arguments: [cs_order_number#16 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(13) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#16] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#15) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(14) Scan parquet spark_catalog.default.catalog_returns +(15) Scan parquet spark_catalog.default.catalog_returns Output [2]: [cr_order_number#18, cr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 6] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [cr_order_number#18, cr_returned_date_sk#19] + +(17) Filter [codegen id : 6] Input [2]: [cr_order_number#18, cr_returned_date_sk#19] +Condition : isnotnull(cr_order_number#18) -(16) Project [codegen id : 6] +(18) Project [codegen id : 6] Output [1]: [cr_order_number#18] Input [2]: [cr_order_number#18, cr_returned_date_sk#19] -(17) Exchange +(19) Exchange Input [1]: [cr_order_number#18] Arguments: hashpartitioning(cr_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) Sort [codegen id : 7] +(20) Sort [codegen id : 7] Input [1]: [cr_order_number#18] Arguments: [cr_order_number#18 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(21) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#18] Join type: LeftAnti Join condition: None -(20) Scan parquet spark_catalog.default.customer_address +(22) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 8] Input [2]: [ca_address_sk#20, ca_state#21] -(22) Filter [codegen id : 8] +(24) Filter [codegen id : 8] Input [2]: [ca_address_sk#20, ca_state#21] Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = GA)) AND isnotnull(ca_address_sk#20)) -(23) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(28) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#20] -(27) Scan parquet spark_catalog.default.call_center +(29) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#22, cc_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 9] Input [2]: [cc_call_center_sk#22, cc_county#23] -(29) Filter [codegen id : 9] +(31) Filter [codegen id : 9] Input [2]: [cc_call_center_sk#22, cc_county#23] Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) -(30) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [1]: [cc_call_center_sk#22] Input [2]: [cc_call_center_sk#22, cc_county#23] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [cc_call_center_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#22] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] -(34) Scan parquet spark_catalog.default.date_dim +(36) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [2]: [d_date_sk#24, d_date#25] -(36) Filter [codegen id : 10] +(38) Filter [codegen id : 10] Input [2]: [d_date_sk#24, d_date#25] Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 2002-02-01)) AND (d_date#25 <= 2002-04-02)) AND isnotnull(d_date_sk#24)) -(37) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#25] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#24] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] Results [3]: [sum#28, sum#29, count#31] -(44) Exchange +(46) Exchange Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] @@ -261,45 +273,45 @@ Results [3]: [count(cs_order_number#5)#30 AS order count #32, MakeDecimal(sum(Un ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -ObjectHashAggregate (52) -+- Exchange (51) - +- ObjectHashAggregate (50) - +- * Project (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet spark_catalog.default.customer_address (46) +ObjectHashAggregate (54) ++- Exchange (53) + +- ObjectHashAggregate (52) + +- * Project (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet spark_catalog.default.customer_address (48) -(46) Scan parquet spark_catalog.default.customer_address +(48) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [2]: [ca_address_sk#20, ca_state#21] -(48) Filter [codegen id : 1] +(50) Filter [codegen id : 1] Input [2]: [ca_address_sk#20, ca_state#21] Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = GA)) AND isnotnull(ca_address_sk#20)) -(49) Project [codegen id : 1] +(51) Project [codegen id : 1] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(50) ObjectHashAggregate +(52) ObjectHashAggregate Input [1]: [ca_address_sk#20] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 17961, 333176, 0, 0)] Aggregate Attributes [1]: [buf#35] Results [1]: [buf#36] -(51) Exchange +(53) Exchange Input [1]: [buf#36] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(52) ObjectHashAggregate +(54) ObjectHashAggregate Input [1]: [buf#36] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 17961, 333176, 0, 0)] @@ -307,45 +319,45 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 1796 Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 17961, 333176, 0, 0)#37 AS bloomFilter#38] Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -ObjectHashAggregate (59) -+- Exchange (58) - +- ObjectHashAggregate (57) - +- * Project (56) - +- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet spark_catalog.default.call_center (53) +ObjectHashAggregate (61) ++- Exchange (60) + +- ObjectHashAggregate (59) + +- * Project (58) + +- * Filter (57) + +- * ColumnarToRow (56) + +- Scan parquet spark_catalog.default.call_center (55) -(53) Scan parquet spark_catalog.default.call_center +(55) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#22, cc_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [2]: [cc_call_center_sk#22, cc_county#23] -(55) Filter [codegen id : 1] +(57) Filter [codegen id : 1] Input [2]: [cc_call_center_sk#22, cc_county#23] Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) -(56) Project [codegen id : 1] +(58) Project [codegen id : 1] Output [1]: [cc_call_center_sk#22] Input [2]: [cc_call_center_sk#22, cc_county#23] -(57) ObjectHashAggregate +(59) ObjectHashAggregate Input [1]: [cc_call_center_sk#22] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(cc_call_center_sk#22, 42), 4, 144, 0, 0)] Aggregate Attributes [1]: [buf#39] Results [1]: [buf#40] -(58) Exchange +(60) Exchange Input [1]: [buf#40] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(59) ObjectHashAggregate +(61) ObjectHashAggregate Input [1]: [buf#40] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(cc_call_center_sk#22, 42), 4, 144, 0, 0)] @@ -353,45 +365,45 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(cc_call_center_sk#22, 42), Results [1]: [bloom_filter_agg(xxhash64(cc_call_center_sk#22, 42), 4, 144, 0, 0)#41 AS bloomFilter#42] Subquery:3 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -ObjectHashAggregate (66) -+- Exchange (65) - +- ObjectHashAggregate (64) - +- * Project (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet spark_catalog.default.date_dim (60) +ObjectHashAggregate (68) ++- Exchange (67) + +- ObjectHashAggregate (66) + +- * Project (65) + +- * Filter (64) + +- * ColumnarToRow (63) + +- Scan parquet spark_catalog.default.date_dim (62) -(60) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_date#25] -(62) Filter [codegen id : 1] +(64) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_date#25] Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 2002-02-01)) AND (d_date#25 <= 2002-04-02)) AND isnotnull(d_date_sk#24)) -(63) Project [codegen id : 1] +(65) Project [codegen id : 1] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#25] -(64) ObjectHashAggregate +(66) ObjectHashAggregate Input [1]: [d_date_sk#24] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(d_date_sk#24, 42), 73049, 1141755, 0, 0)] Aggregate Attributes [1]: [buf#43] Results [1]: [buf#44] -(65) Exchange +(67) Exchange Input [1]: [buf#44] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(66) ObjectHashAggregate +(68) ObjectHashAggregate Input [1]: [buf#44] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(d_date_sk#24, 42), 73049, 1141755, 0, 0)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index def1677f94401..2c6a10e532f17 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 17961, 333176, 0, 0),bloomFilter,buf] Exchange #3 @@ -65,9 +65,10 @@ WholeStageCodegen (12) Exchange [cs_order_number] #6 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + Filter [cs_order_number,cs_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] InputAdapter WholeStageCodegen (7) Sort [cr_order_number] @@ -75,9 +76,10 @@ WholeStageCodegen (12) Exchange [cr_order_number] #7 WholeStageCodegen (6) Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + Filter [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 70a5c5bed4e9c..796995db700ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,56 +1,58 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * SortMergeJoin LeftAnti (21) + : : : :- * Project (14) + : : : : +- * SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet spark_catalog.default.catalog_returns (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet spark_catalog.default.date_dim (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.call_center (34) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet spark_catalog.default.catalog_returns (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet spark_catalog.default.date_dim (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet spark_catalog.default.call_center (36) (1) Scan parquet spark_catalog.default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -58,7 +60,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) +Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -76,182 +78,192 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] +Condition : (isnotnull(cs_order_number#10) AND isnotnull(cs_warehouse_sk#9)) + +(10) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#9, cs_order_number#10] Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -(10) Exchange +(11) Exchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(13) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#10] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(14) Scan parquet spark_catalog.default.catalog_returns +(15) Scan parquet spark_catalog.default.catalog_returns Output [2]: [cr_order_number#12, cr_returned_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 6] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [cr_order_number#12, cr_returned_date_sk#13] + +(17) Filter [codegen id : 6] Input [2]: [cr_order_number#12, cr_returned_date_sk#13] +Condition : isnotnull(cr_order_number#12) -(16) Project [codegen id : 6] +(18) Project [codegen id : 6] Output [1]: [cr_order_number#12] Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -(17) Exchange +(19) Exchange Input [1]: [cr_order_number#12] Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) Sort [codegen id : 7] +(20) Sort [codegen id : 7] Input [1]: [cr_order_number#12] Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(21) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join type: LeftAnti Join condition: None -(20) Scan parquet spark_catalog.default.date_dim +(22) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 8] Input [2]: [d_date_sk#14, d_date#15] -(22) Filter [codegen id : 8] +(24) Filter [codegen id : 8] Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) -(23) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [1]: [d_date_sk#14] Input [2]: [d_date_sk#14, d_date#15] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(28) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -(27) Scan parquet spark_catalog.default.customer_address +(29) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#16, ca_state#17] -(29) Filter [codegen id : 9] +(31) Filter [codegen id : 9] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) -(30) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -(34) Scan parquet spark_catalog.default.call_center +(36) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#18, cc_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [2]: [cc_call_center_sk#18, cc_county#19] -(36) Filter [codegen id : 10] +(38) Filter [codegen id : 10] Input [2]: [cc_call_center_sk#18, cc_county#19] Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) -(37) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [1]: [cc_call_center_sk#18] Input [2]: [cc_call_center_sk#18, cc_county#19] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [cc_call_center_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(46) Exchange Input [3]: [sum#22, sum#23, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 0ae1584bd9ccd..a2022d0bff800 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,9 +35,10 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + Filter [cs_order_number,cs_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] InputAdapter WholeStageCodegen (7) Sort [cr_order_number] @@ -45,9 +46,10 @@ WholeStageCodegen (12) Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + Filter [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index 4114749e28d80..ec8332281359f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -1,94 +1,96 @@ == Physical Plan == -* HashAggregate (90) -+- Exchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (48) - : : +- * SortMergeJoin LeftSemi (47) - : : :- * Sort (28) - : : : +- Exchange (27) - : : : +- * Project (26) - : : : +- * SortMergeJoin LeftSemi (25) - : : : :- * Sort (4) - : : : : +- Exchange (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- * Sort (24) - : : : +- * Project (23) - : : : +- * Filter (22) - : : : +- * HashAggregate (21) - : : : +- * HashAggregate (20) - : : : +- * Project (19) - : : : +- * SortMergeJoin Inner (18) - : : : :- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : : +- ReusedExchange (8) - : : : +- * Sort (17) - : : : +- Exchange (16) - : : : +- * Filter (15) - : : : +- * ColumnarToRow (14) - : : : +- Scan parquet spark_catalog.default.item (13) - : : +- * Sort (46) - : : +- * Project (45) - : : +- * Filter (44) - : : +- * HashAggregate (43) - : : +- * HashAggregate (42) - : : +- * Project (41) - : : +- * SortMergeJoin Inner (40) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * Filter (31) - : : : +- * ColumnarToRow (30) - : : : +- Scan parquet spark_catalog.default.store_sales (29) - : : +- * Sort (39) - : : +- Exchange (38) - : : +- * Filter (37) - : : +- * ColumnarToRow (36) - : : +- Scan parquet spark_catalog.default.customer (35) - : +- ReusedExchange (49) - +- * Project (86) - +- * BroadcastHashJoin Inner BuildRight (85) - :- * Project (83) - : +- * SortMergeJoin LeftSemi (82) - : :- * Sort (70) - : : +- Exchange (69) - : : +- * Project (68) - : : +- * SortMergeJoin LeftSemi (67) - : : :- * Sort (55) - : : : +- Exchange (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet spark_catalog.default.web_sales (52) - : : +- * Sort (66) - : : +- * Project (65) - : : +- * Filter (64) - : : +- * HashAggregate (63) - : : +- * HashAggregate (62) - : : +- * Project (61) - : : +- * SortMergeJoin Inner (60) - : : :- * Sort (57) - : : : +- ReusedExchange (56) - : : +- * Sort (59) - : : +- ReusedExchange (58) - : +- * Sort (81) - : +- * Project (80) - : +- * Filter (79) - : +- * HashAggregate (78) - : +- * HashAggregate (77) - : +- * Project (76) - : +- * SortMergeJoin Inner (75) - : :- * Sort (72) - : : +- ReusedExchange (71) - : +- * Sort (74) - : +- ReusedExchange (73) - +- ReusedExchange (84) +* HashAggregate (92) ++- Exchange (91) + +- * HashAggregate (90) + +- Union (89) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * SortMergeJoin LeftSemi (48) + : : :- * Sort (29) + : : : +- Exchange (28) + : : : +- * Project (27) + : : : +- * SortMergeJoin LeftSemi (26) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) + : : : +- * Sort (25) + : : : +- * Project (24) + : : : +- * Filter (23) + : : : +- * HashAggregate (22) + : : : +- * HashAggregate (21) + : : : +- * Project (20) + : : : +- * SortMergeJoin Inner (19) + : : : :- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * Filter (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet spark_catalog.default.item (14) + : : +- * Sort (47) + : : +- * Project (46) + : : +- * Filter (45) + : : +- * HashAggregate (44) + : : +- * HashAggregate (43) + : : +- * Project (42) + : : +- * SortMergeJoin Inner (41) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * Project (33) + : : : +- * Filter (32) + : : : +- * ColumnarToRow (31) + : : : +- Scan parquet spark_catalog.default.store_sales (30) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * Filter (38) + : : +- * ColumnarToRow (37) + : : +- Scan parquet spark_catalog.default.customer (36) + : +- ReusedExchange (50) + +- * Project (88) + +- * BroadcastHashJoin Inner BuildRight (87) + :- * Project (85) + : +- * SortMergeJoin LeftSemi (84) + : :- * Sort (72) + : : +- Exchange (71) + : : +- * Project (70) + : : +- * SortMergeJoin LeftSemi (69) + : : :- * Sort (57) + : : : +- Exchange (56) + : : : +- * Filter (55) + : : : +- * ColumnarToRow (54) + : : : +- Scan parquet spark_catalog.default.web_sales (53) + : : +- * Sort (68) + : : +- * Project (67) + : : +- * Filter (66) + : : +- * HashAggregate (65) + : : +- * HashAggregate (64) + : : +- * Project (63) + : : +- * SortMergeJoin Inner (62) + : : :- * Sort (59) + : : : +- ReusedExchange (58) + : : +- * Sort (61) + : : +- ReusedExchange (60) + : +- * Sort (83) + : +- * Project (82) + : +- * Filter (81) + : +- * HashAggregate (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * SortMergeJoin Inner (77) + : :- * Sort (74) + : : +- ReusedExchange (73) + : +- * Sort (76) + : +- ReusedExchange (75) + +- ReusedExchange (86) (1) Scan parquet spark_catalog.default.catalog_sales @@ -96,20 +98,25 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Exchange +(3) Filter [codegen id : 1] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) + +(4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(4) Sort [codegen id : 2] +(5) Sort [codegen id : 2] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(5) Scan parquet spark_catalog.default.store_sales +(6) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -117,390 +124,395 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 4] +(7) ColumnarToRow [codegen id : 4] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(7) Filter [codegen id : 4] +(8) Filter [codegen id : 4] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(8) ReusedExchange [Reuses operator id: 100] +(9) ReusedExchange [Reuses operator id: 102] Output [2]: [d_date_sk#10, d_date#11] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -(11) Exchange +(12) Exchange Input [2]: [ss_item_sk#7, d_date#11] Arguments: hashpartitioning(ss_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [2]: [ss_item_sk#7, d_date#11] Arguments: [ss_item_sk#7 ASC NULLS FIRST], false, 0 -(13) Scan parquet spark_catalog.default.item +(14) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#12, i_item_desc#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 6] +(15) ColumnarToRow [codegen id : 6] Input [2]: [i_item_sk#12, i_item_desc#13] -(15) Filter [codegen id : 6] +(16) Filter [codegen id : 6] Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(16) Exchange +(17) Exchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: hashpartitioning(i_item_sk#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(17) Sort [codegen id : 7] +(18) Sort [codegen id : 7] Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: [i_item_sk#12 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin [codegen id : 8] +(19) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 8] +(20) Project [codegen id : 8] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(20) HashAggregate [codegen id : 8] +(21) HashAggregate [codegen id : 8] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(21) HashAggregate [codegen id : 8] +(22) HashAggregate [codegen id : 8] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(22) Filter [codegen id : 8] +(23) Filter [codegen id : 8] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(23) Project [codegen id : 8] +(24) Project [codegen id : 8] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(24) Sort [codegen id : 8] +(25) Sort [codegen id : 8] Input [1]: [item_sk#18] Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin [codegen id : 9] +(26) SortMergeJoin [codegen id : 9] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(26) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(27) Exchange +(28) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) Sort [codegen id : 10] +(29) Sort [codegen id : 10] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(29) Scan parquet spark_catalog.default.store_sales +(30) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 11] +(31) ColumnarToRow [codegen id : 11] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -(31) Filter [codegen id : 11] +(32) Filter [codegen id : 11] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(32) Project [codegen id : 11] +(33) Project [codegen id : 11] Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -(33) Exchange +(34) Exchange Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] Arguments: hashpartitioning(ss_customer_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(34) Sort [codegen id : 12] +(35) Sort [codegen id : 12] Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] Arguments: [ss_customer_sk#20 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.customer +(36) Scan parquet spark_catalog.default.customer Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 13] +(37) ColumnarToRow [codegen id : 13] Input [1]: [c_customer_sk#24] -(37) Filter [codegen id : 13] +(38) Filter [codegen id : 13] Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(38) Exchange +(39) Exchange Input [1]: [c_customer_sk#24] Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) Sort [codegen id : 14] +(40) Sort [codegen id : 14] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(40) SortMergeJoin [codegen id : 15] +(41) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 15] +(42) Project [codegen id : 15] Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(42) HashAggregate [codegen id : 15] +(43) HashAggregate [codegen id : 15] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(43) HashAggregate [codegen id : 15] +(44) HashAggregate [codegen id : 15] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(44) Filter [codegen id : 15] +(45) Filter [codegen id : 15] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(45) Project [codegen id : 15] +(46) Project [codegen id : 15] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(46) Sort [codegen id : 15] +(47) Sort [codegen id : 15] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(47) SortMergeJoin [codegen id : 17] +(48) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(48) Project [codegen id : 17] +(49) Project [codegen id : 17] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(49) ReusedExchange [Reuses operator id: 95] +(50) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#33] -(50) BroadcastHashJoin [codegen id : 17] +(51) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(51) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] -(52) Scan parquet spark_catalog.default.web_sales +(53) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 18] +(54) ColumnarToRow [codegen id : 18] +Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] + +(55) Filter [codegen id : 18] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Condition : (isnotnull(ws_item_sk#35) AND isnotnull(ws_bill_customer_sk#36)) -(54) Exchange +(56) Exchange Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(55) Sort [codegen id : 19] +(57) Sort [codegen id : 19] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 -(56) ReusedExchange [Reuses operator id: 11] +(58) ReusedExchange [Reuses operator id: 12] Output [2]: [ss_item_sk#7, d_date#11] -(57) Sort [codegen id : 22] +(59) Sort [codegen id : 22] Input [2]: [ss_item_sk#7, d_date#11] Arguments: [ss_item_sk#7 ASC NULLS FIRST], false, 0 -(58) ReusedExchange [Reuses operator id: 16] +(60) ReusedExchange [Reuses operator id: 17] Output [2]: [i_item_sk#12, i_item_desc#13] -(59) Sort [codegen id : 24] +(61) Sort [codegen id : 24] Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: [i_item_sk#12 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 25] +(62) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(61) Project [codegen id : 25] +(63) Project [codegen id : 25] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(62) HashAggregate [codegen id : 25] +(64) HashAggregate [codegen id : 25] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(63) HashAggregate [codegen id : 25] +(65) HashAggregate [codegen id : 25] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(64) Filter [codegen id : 25] +(66) Filter [codegen id : 25] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(65) Project [codegen id : 25] +(67) Project [codegen id : 25] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(66) Sort [codegen id : 25] +(68) Sort [codegen id : 25] Input [1]: [item_sk#18] Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin [codegen id : 26] +(69) SortMergeJoin [codegen id : 26] Left keys [1]: [ws_item_sk#35] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(68) Project [codegen id : 26] +(70) Project [codegen id : 26] Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(69) Exchange +(71) Exchange Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(70) Sort [codegen id : 27] +(72) Sort [codegen id : 27] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 -(71) ReusedExchange [Reuses operator id: 33] +(73) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(72) Sort [codegen id : 29] +(74) Sort [codegen id : 29] Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] Arguments: [ss_customer_sk#20 ASC NULLS FIRST], false, 0 -(73) ReusedExchange [Reuses operator id: 38] +(75) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#24] -(74) Sort [codegen id : 31] +(76) Sort [codegen id : 31] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 32] +(77) SortMergeJoin [codegen id : 32] Left keys [1]: [ss_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(76) Project [codegen id : 32] +(78) Project [codegen id : 32] Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(77) HashAggregate [codegen id : 32] +(79) HashAggregate [codegen id : 32] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(78) HashAggregate [codegen id : 32] +(80) HashAggregate [codegen id : 32] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(79) Filter [codegen id : 32] +(81) Filter [codegen id : 32] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(80) Project [codegen id : 32] +(82) Project [codegen id : 32] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(81) Sort [codegen id : 32] +(83) Sort [codegen id : 32] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(82) SortMergeJoin [codegen id : 34] +(84) SortMergeJoin [codegen id : 34] Left keys [1]: [ws_bill_customer_sk#36] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(83) Project [codegen id : 34] +(85) Project [codegen id : 34] Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(84) ReusedExchange [Reuses operator id: 95] +(86) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#40] -(85) BroadcastHashJoin [codegen id : 34] +(87) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#39] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(86) Project [codegen id : 34] +(88) Project [codegen id : 34] Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#41] Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#40] -(87) Union +(89) Union -(88) HashAggregate [codegen id : 35] +(90) HashAggregate [codegen id : 35] Input [1]: [sales#34] Keys: [] Functions [1]: [partial_sum(sales#34)] Aggregate Attributes [2]: [sum#42, isEmpty#43] Results [2]: [sum#44, isEmpty#45] -(89) Exchange +(91) Exchange Input [2]: [sum#44, isEmpty#45] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(90) HashAggregate [codegen id : 36] +(92) HashAggregate [codegen id : 36] Input [2]: [sum#44, isEmpty#45] Keys: [] Functions [1]: [sum(sales#34)] @@ -510,86 +522,86 @@ Results [1]: [sum(sales#34)#46 AS sum(sales)#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * Project (94) - +- * Filter (93) - +- * ColumnarToRow (92) - +- Scan parquet spark_catalog.default.date_dim (91) +BroadcastExchange (97) ++- * Project (96) + +- * Filter (95) + +- * ColumnarToRow (94) + +- Scan parquet spark_catalog.default.date_dim (93) -(91) Scan parquet spark_catalog.default.date_dim +(93) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#33, d_year#48, d_moy#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 1] +(94) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#33, d_year#48, d_moy#49] -(93) Filter [codegen id : 1] +(95) Filter [codegen id : 1] Input [3]: [d_date_sk#33, d_year#48, d_moy#49] Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2000)) AND (d_moy#49 = 2)) AND isnotnull(d_date_sk#33)) -(94) Project [codegen id : 1] +(96) Project [codegen id : 1] Output [1]: [d_date_sk#33] Input [3]: [d_date_sk#33, d_year#48, d_moy#49] -(95) BroadcastExchange +(97) BroadcastExchange Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (100) -+- * Project (99) - +- * Filter (98) - +- * ColumnarToRow (97) - +- Scan parquet spark_catalog.default.date_dim (96) +Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (102) ++- * Project (101) + +- * Filter (100) + +- * ColumnarToRow (99) + +- Scan parquet spark_catalog.default.date_dim (98) -(96) Scan parquet spark_catalog.default.date_dim +(98) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 1] +(99) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#50] -(98) Filter [codegen id : 1] +(100) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#50] Condition : (d_year#50 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(99) Project [codegen id : 1] +(101) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#50] -(100) BroadcastExchange +(102) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (117) -+- Exchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * HashAggregate (113) - +- * Project (112) - +- * SortMergeJoin Inner (111) - :- * Sort (108) - : +- Exchange (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet spark_catalog.default.store_sales (101) - : +- ReusedExchange (104) - +- * Sort (110) - +- ReusedExchange (109) - - -(101) Scan parquet spark_catalog.default.store_sales +Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (119) ++- Exchange (118) + +- * HashAggregate (117) + +- * HashAggregate (116) + +- * HashAggregate (115) + +- * Project (114) + +- * SortMergeJoin Inner (113) + :- * Sort (110) + : +- Exchange (109) + : +- * Project (108) + : +- * BroadcastHashJoin Inner BuildRight (107) + : :- * Filter (105) + : : +- * ColumnarToRow (104) + : : +- Scan parquet spark_catalog.default.store_sales (103) + : +- ReusedExchange (106) + +- * Sort (112) + +- ReusedExchange (111) + + +(103) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] @@ -597,115 +609,115 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] +(104) ColumnarToRow [codegen id : 2] Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -(103) Filter [codegen id : 2] +(105) Filter [codegen id : 2] Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] Condition : isnotnull(ss_customer_sk#51) -(104) ReusedExchange [Reuses operator id: 122] +(106) ReusedExchange [Reuses operator id: 124] Output [1]: [d_date_sk#56] -(105) BroadcastHashJoin [codegen id : 2] +(107) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#54] Right keys [1]: [d_date_sk#56] Join type: Inner Join condition: None -(106) Project [codegen id : 2] +(108) Project [codegen id : 2] Output [3]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53] Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, d_date_sk#56] -(107) Exchange +(109) Exchange Input [3]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53] Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(108) Sort [codegen id : 3] +(110) Sort [codegen id : 3] Input [3]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53] Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(109) ReusedExchange [Reuses operator id: 38] +(111) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#57] -(110) Sort [codegen id : 5] +(112) Sort [codegen id : 5] Input [1]: [c_customer_sk#57] Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0 -(111) SortMergeJoin [codegen id : 6] +(113) SortMergeJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#51] Right keys [1]: [c_customer_sk#57] Join type: Inner Join condition: None -(112) Project [codegen id : 6] +(114) Project [codegen id : 6] Output [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#57] Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, c_customer_sk#57] -(113) HashAggregate [codegen id : 6] +(115) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#57] Keys [1]: [c_customer_sk#57] Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] Aggregate Attributes [2]: [sum#58, isEmpty#59] Results [3]: [c_customer_sk#57, sum#60, isEmpty#61] -(114) HashAggregate [codegen id : 6] +(116) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#57, sum#60, isEmpty#61] Keys [1]: [c_customer_sk#57] Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] Aggregate Attributes [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62] Results [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62 AS csales#63] -(115) HashAggregate [codegen id : 6] +(117) HashAggregate [codegen id : 6] Input [1]: [csales#63] Keys: [] Functions [1]: [partial_max(csales#63)] Aggregate Attributes [1]: [max#64] Results [1]: [max#65] -(116) Exchange +(118) Exchange Input [1]: [max#65] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(117) HashAggregate [codegen id : 7] +(119) HashAggregate [codegen id : 7] Input [1]: [max#65] Keys: [] Functions [1]: [max(csales#63)] Aggregate Attributes [1]: [max(csales#63)#66] Results [1]: [max(csales#63)#66 AS tpcds_cmax#67] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 -BroadcastExchange (122) -+- * Project (121) - +- * Filter (120) - +- * ColumnarToRow (119) - +- Scan parquet spark_catalog.default.date_dim (118) +Subquery:4 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 +BroadcastExchange (124) ++- * Project (123) + +- * Filter (122) + +- * ColumnarToRow (121) + +- Scan parquet spark_catalog.default.date_dim (120) -(118) Scan parquet spark_catalog.default.date_dim +(120) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#56, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(119) ColumnarToRow [codegen id : 1] +(121) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#56, d_year#68] -(120) Filter [codegen id : 1] +(122) Filter [codegen id : 1] Input [2]: [d_date_sk#56, d_year#68] Condition : (d_year#68 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#56)) -(121) Project [codegen id : 1] +(123) Project [codegen id : 1] Output [1]: [d_date_sk#56] Input [2]: [d_date_sk#56, d_year#68] -(122) BroadcastExchange +(124) BroadcastExchange Input [1]: [d_date_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index 1e863dd5b1b43..3cc9e43d35ecb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -25,17 +25,18 @@ WholeStageCodegen (36) InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [cs_item_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter WholeStageCodegen (8) Sort [item_sk] @@ -166,10 +167,11 @@ WholeStageCodegen (36) InputAdapter Exchange [ws_item_sk] #14 WholeStageCodegen (18) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_item_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (25) Sort [item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 74dcb73f33a98..d4a86c1d22906 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,70 +1,72 @@ == Physical Plan == -* HashAggregate (66) -+- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * SortMergeJoin LeftSemi (41) - : : :- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (21) - : : : :- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- Exchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * Filter (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- Scan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet spark_catalog.default.item (9) - : : +- * Sort (40) - : : +- * Project (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- Exchange (36) - : : +- * HashAggregate (35) - : : +- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (28) - : : : +- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.store_sales (25) - : : +- BroadcastExchange (32) - : : +- * Filter (31) - : : +- * ColumnarToRow (30) - : : +- Scan parquet spark_catalog.default.customer (29) - : +- ReusedExchange (43) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * SortMergeJoin LeftSemi (58) - : :- * Sort (52) - : : +- Exchange (51) - : : +- * Project (50) - : : +- * BroadcastHashJoin LeftSemi BuildRight (49) - : : :- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (48) - : +- * Sort (57) - : +- * Project (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) +* HashAggregate (68) ++- Exchange (67) + +- * HashAggregate (66) + +- Union (65) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * SortMergeJoin LeftSemi (42) + : : :- * Sort (25) + : : : +- Exchange (24) + : : : +- * Project (23) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- Exchange (17) + : : : +- * HashAggregate (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (13) + : : : +- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet spark_catalog.default.item (10) + : : +- * Sort (41) + : : +- * Project (40) + : : +- * Filter (39) + : : +- * HashAggregate (38) + : : +- Exchange (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Project (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet spark_catalog.default.store_sales (26) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet spark_catalog.default.customer (30) + : +- ReusedExchange (44) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * SortMergeJoin LeftSemi (60) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * Project (52) + : : +- * BroadcastHashJoin LeftSemi BuildRight (51) + : : :- * Filter (49) + : : : +- * ColumnarToRow (48) + : : : +- Scan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (50) + : +- * Sort (59) + : +- * Project (58) + : +- * Filter (57) + : +- * HashAggregate (56) + : +- ReusedExchange (55) + +- ReusedExchange (62) (1) Scan parquet spark_catalog.default.catalog_sales @@ -72,12 +74,17 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Scan parquet spark_catalog.default.store_sales +(3) Filter [codegen id : 5] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) + +(4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -85,291 +92,296 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) ColumnarToRow [codegen id : 3] +(5) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(5) Filter [codegen id : 3] +(6) Filter [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) ReusedExchange [Reuses operator id: 76] +(7) ReusedExchange [Reuses operator id: 78] Output [2]: [d_date_sk#10, d_date#11] -(7) BroadcastHashJoin [codegen id : 3] +(8) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(8) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -(9) Scan parquet spark_catalog.default.item +(10) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#12, i_item_desc#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 2] +(11) ColumnarToRow [codegen id : 2] Input [2]: [i_item_sk#12, i_item_desc#13] -(11) Filter [codegen id : 2] +(12) Filter [codegen id : 2] Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(12) BroadcastExchange +(13) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(13) BroadcastHashJoin [codegen id : 3] +(14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(15) Project [codegen id : 3] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(15) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 3] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(16) Exchange +(17) Exchange Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 4] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(18) Filter [codegen id : 4] +(19) Filter [codegen id : 4] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(19) Project [codegen id : 4] +(20) Project [codegen id : 4] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(20) BroadcastExchange +(21) BroadcastExchange Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(22) Project [codegen id : 5] +(23) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(23) Exchange +(24) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(24) Sort [codegen id : 6] +(25) Sort [codegen id : 6] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(25) Scan parquet spark_catalog.default.store_sales +(26) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(27) ColumnarToRow [codegen id : 8] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -(27) Filter [codegen id : 8] +(28) Filter [codegen id : 8] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(28) Project [codegen id : 8] +(29) Project [codegen id : 8] Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -(29) Scan parquet spark_catalog.default.customer +(30) Scan parquet spark_catalog.default.customer Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 7] Input [1]: [c_customer_sk#24] -(31) Filter [codegen id : 7] +(32) Filter [codegen id : 7] Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(32) BroadcastExchange +(33) BroadcastExchange Input [1]: [c_customer_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(35) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 8] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(36) Exchange +(37) Exchange Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 9] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(38) Filter [codegen id : 9] +(39) Filter [codegen id : 9] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(39) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(40) Sort [codegen id : 9] +(41) Sort [codegen id : 9] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 11] +(42) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(42) Project [codegen id : 11] +(43) Project [codegen id : 11] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(43) ReusedExchange [Reuses operator id: 71] +(44) ReusedExchange [Reuses operator id: 73] Output [1]: [d_date_sk#33] -(44) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(45) Project [codegen id : 11] +(46) Project [codegen id : 11] Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] -(46) Scan parquet spark_catalog.default.web_sales +(47) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] +(48) ColumnarToRow [codegen id : 16] +Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] + +(49) Filter [codegen id : 16] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Condition : (isnotnull(ws_item_sk#35) AND isnotnull(ws_bill_customer_sk#36)) -(48) ReusedExchange [Reuses operator id: 20] +(50) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#18] -(49) BroadcastHashJoin [codegen id : 16] +(51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#35] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(50) Project [codegen id : 16] +(52) Project [codegen id : 16] Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(51) Exchange +(53) Exchange Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(52) Sort [codegen id : 17] +(54) Sort [codegen id : 17] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 -(53) ReusedExchange [Reuses operator id: 36] +(55) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(54) HashAggregate [codegen id : 20] +(56) HashAggregate [codegen id : 20] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(55) Filter [codegen id : 20] +(57) Filter [codegen id : 20] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(56) Project [codegen id : 20] +(58) Project [codegen id : 20] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(57) Sort [codegen id : 20] +(59) Sort [codegen id : 20] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 22] +(60) SortMergeJoin [codegen id : 22] Left keys [1]: [ws_bill_customer_sk#36] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(59) Project [codegen id : 22] +(61) Project [codegen id : 22] Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(60) ReusedExchange [Reuses operator id: 71] +(62) ReusedExchange [Reuses operator id: 73] Output [1]: [d_date_sk#40] -(61) BroadcastHashJoin [codegen id : 22] +(63) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#39] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(62) Project [codegen id : 22] +(64) Project [codegen id : 22] Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#41] Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#40] -(63) Union +(65) Union -(64) HashAggregate [codegen id : 23] +(66) HashAggregate [codegen id : 23] Input [1]: [sales#34] Keys: [] Functions [1]: [partial_sum(sales#34)] Aggregate Attributes [2]: [sum#42, isEmpty#43] Results [2]: [sum#44, isEmpty#45] -(65) Exchange +(67) Exchange Input [2]: [sum#44, isEmpty#45] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(66) HashAggregate [codegen id : 24] +(68) HashAggregate [codegen id : 24] Input [2]: [sum#44, isEmpty#45] Keys: [] Functions [1]: [sum(sales#34)] @@ -379,84 +391,84 @@ Results [1]: [sum(sales#34)#46 AS sum(sales)#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (71) -+- * Project (70) - +- * Filter (69) - +- * ColumnarToRow (68) - +- Scan parquet spark_catalog.default.date_dim (67) +BroadcastExchange (73) ++- * Project (72) + +- * Filter (71) + +- * ColumnarToRow (70) + +- Scan parquet spark_catalog.default.date_dim (69) -(67) Scan parquet spark_catalog.default.date_dim +(69) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#33, d_year#48, d_moy#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 1] +(70) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#33, d_year#48, d_moy#49] -(69) Filter [codegen id : 1] +(71) Filter [codegen id : 1] Input [3]: [d_date_sk#33, d_year#48, d_moy#49] Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2000)) AND (d_moy#49 = 2)) AND isnotnull(d_date_sk#33)) -(70) Project [codegen id : 1] +(72) Project [codegen id : 1] Output [1]: [d_date_sk#33] Input [3]: [d_date_sk#33, d_year#48, d_moy#49] -(71) BroadcastExchange +(73) BroadcastExchange Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * Project (75) - +- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet spark_catalog.default.date_dim (72) +Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (78) ++- * Project (77) + +- * Filter (76) + +- * ColumnarToRow (75) + +- Scan parquet spark_catalog.default.date_dim (74) -(72) Scan parquet spark_catalog.default.date_dim +(74) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#50] -(74) Filter [codegen id : 1] +(76) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#50] Condition : (d_year#50 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(75) Project [codegen id : 1] +(77) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#50] -(76) BroadcastExchange +(78) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (91) -+- Exchange (90) - +- * HashAggregate (89) - +- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * BroadcastHashJoin Inner BuildRight (84) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Filter (79) - : : +- * ColumnarToRow (78) - : : +- Scan parquet spark_catalog.default.store_sales (77) - : +- ReusedExchange (80) - +- ReusedExchange (83) - - -(77) Scan parquet spark_catalog.default.store_sales +Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (93) ++- Exchange (92) + +- * HashAggregate (91) + +- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- * Project (87) + +- * BroadcastHashJoin Inner BuildRight (86) + :- * Project (84) + : +- * BroadcastHashJoin Inner BuildRight (83) + : :- * Filter (81) + : : +- * ColumnarToRow (80) + : : +- Scan parquet spark_catalog.default.store_sales (79) + : +- ReusedExchange (82) + +- ReusedExchange (85) + + +(79) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] @@ -464,107 +476,107 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(78) ColumnarToRow [codegen id : 3] +(80) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -(79) Filter [codegen id : 3] +(81) Filter [codegen id : 3] Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] Condition : isnotnull(ss_customer_sk#51) -(80) ReusedExchange [Reuses operator id: 32] +(82) ReusedExchange [Reuses operator id: 33] Output [1]: [c_customer_sk#56] -(81) BroadcastHashJoin [codegen id : 3] +(83) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#51] Right keys [1]: [c_customer_sk#56] Join type: Inner Join condition: None -(82) Project [codegen id : 3] +(84) Project [codegen id : 3] Output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -(83) ReusedExchange [Reuses operator id: 96] +(85) ReusedExchange [Reuses operator id: 98] Output [1]: [d_date_sk#57] -(84) BroadcastHashJoin [codegen id : 3] +(86) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#54] Right keys [1]: [d_date_sk#57] Join type: Inner Join condition: None -(85) Project [codegen id : 3] +(87) Project [codegen id : 3] Output [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] -(86) HashAggregate [codegen id : 3] +(88) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] Keys [1]: [c_customer_sk#56] Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] Aggregate Attributes [2]: [sum#58, isEmpty#59] Results [3]: [c_customer_sk#56, sum#60, isEmpty#61] -(87) Exchange +(89) Exchange Input [3]: [c_customer_sk#56, sum#60, isEmpty#61] Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(88) HashAggregate [codegen id : 4] +(90) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#56, sum#60, isEmpty#61] Keys [1]: [c_customer_sk#56] Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] Aggregate Attributes [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62] Results [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62 AS csales#63] -(89) HashAggregate [codegen id : 4] +(91) HashAggregate [codegen id : 4] Input [1]: [csales#63] Keys: [] Functions [1]: [partial_max(csales#63)] Aggregate Attributes [1]: [max#64] Results [1]: [max#65] -(90) Exchange +(92) Exchange Input [1]: [max#65] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(91) HashAggregate [codegen id : 5] +(93) HashAggregate [codegen id : 5] Input [1]: [max#65] Keys: [] Functions [1]: [max(csales#63)] Aggregate Attributes [1]: [max(csales#63)#66] Results [1]: [max(csales#63)#66 AS tpcds_cmax#67] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 -BroadcastExchange (96) -+- * Project (95) - +- * Filter (94) - +- * ColumnarToRow (93) - +- Scan parquet spark_catalog.default.date_dim (92) +Subquery:4 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 +BroadcastExchange (98) ++- * Project (97) + +- * Filter (96) + +- * ColumnarToRow (95) + +- Scan parquet spark_catalog.default.date_dim (94) -(92) Scan parquet spark_catalog.default.date_dim +(94) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#57, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(93) ColumnarToRow [codegen id : 1] +(95) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#57, d_year#68] -(94) Filter [codegen id : 1] +(96) Filter [codegen id : 1] Input [2]: [d_date_sk#57, d_year#68] Condition : (d_year#68 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) -(95) Project [codegen id : 1] +(97) Project [codegen id : 1] Output [1]: [d_date_sk#57] Input [2]: [d_date_sk#57, d_year#68] -(96) BroadcastExchange +(98) BroadcastExchange Input [1]: [d_date_sk#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 089c424d66911..6d3a9dd3f3b77 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -19,17 +19,18 @@ WholeStageCodegen (24) WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [cs_item_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -136,10 +137,11 @@ WholeStageCodegen (24) WholeStageCodegen (16) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_item_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [item_sk] #4 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 1c47d62c6ac89..f5a2d52b1d900 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -135,7 +135,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -143,7 +143,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) +Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) (4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -479,7 +479,7 @@ Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] @@ -487,7 +487,7 @@ Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price (76) Filter [codegen id : 28] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_customer_sk#44) +Condition : (isnotnull(ws_bill_customer_sk#44) AND isnotnull(ws_item_sk#43)) (77) Exchange Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index 5789b1293678a..2b137f14d458c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_bill_customer_sk] + Filter [cs_bill_customer_sk,cs_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -210,7 +210,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [ws_item_sk] #16 WholeStageCodegen (28) - Filter [ws_bill_customer_sk] + Filter [ws_bill_customer_sk,ws_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 479bb62b73e58..f70d8336ca3e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -93,7 +93,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] @@ -101,7 +101,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) +Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] @@ -389,7 +389,7 @@ Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 23] @@ -397,7 +397,7 @@ Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price (65) Filter [codegen id : 23] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_customer_sk#44) +Condition : (isnotnull(ws_bill_customer_sk#44) AND isnotnull(ws_item_sk#43)) (66) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 016630e2a88b0..5143023afdc62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] + Filter [cs_bill_customer_sk,cs_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -166,7 +166,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (23) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] + Filter [ws_bill_customer_sk,ws_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 075a67dbbaaf6..dc2c0e50fa26d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -96,7 +96,7 @@ Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, Output [2]: [i_item_sk#7, i_manufact_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] @@ -104,13 +104,13 @@ Input [2]: [i_item_sk#7, i_manufact_id#8] (9) Filter [codegen id : 3] Input [2]: [i_item_sk#7, i_manufact_id#8] -Condition : isnotnull(i_item_sk#7) +Condition : (isnotnull(i_item_sk#7) AND isnotnull(i_manufact_id#8)) (10) Scan parquet spark_catalog.default.item Output [2]: [i_category#9, i_manufact_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -118,7 +118,7 @@ Input [2]: [i_category#9, i_manufact_id#10] (12) Filter [codegen id : 2] Input [2]: [i_category#9, i_manufact_id#10] -Condition : (isnotnull(i_category#9) AND (i_category#9 = Electronics )) +Condition : ((isnotnull(i_category#9) AND (i_category#9 = Electronics )) AND isnotnull(i_manufact_id#10)) (13) Project [codegen id : 2] Output [1]: [i_manufact_id#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index e47174615570c..ce831cb030223 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #4 WholeStageCodegen (3) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + Filter [i_item_sk,i_manufact_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (2) Project [i_manufact_id] - Filter [i_category] + Filter [i_category,i_manufact_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index eae0ce1329922..f1dfd00dd1a38 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_manufact_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,13 +136,13 @@ Input [2]: [i_item_sk#9, i_manufact_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) +Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_manufact_id#10)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -150,7 +150,7 @@ Input [2]: [i_category#11, i_manufact_id#12] (19) Filter [codegen id : 3] Input [2]: [i_category#11, i_manufact_id#12] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics )) +Condition : ((isnotnull(i_category#11) AND (i_category#11 = Electronics )) AND isnotnull(i_manufact_id#12)) (20) Project [codegen id : 3] Output [1]: [i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 959f7932eb5ae..3e322bd8c0c07 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + Filter [i_item_sk,i_manufact_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category] + Filter [i_category,i_manufact_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 414da85ba6ef0..f68ecc12142bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -1,64 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * HashAggregate (52) - +- Exchange (51) - +- * HashAggregate (50) - +- * Project (49) - +- * SortMergeJoin Inner (48) - :- * Sort (42) - : +- Exchange (41) - : +- * Project (40) - : +- * SortMergeJoin Inner (39) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * SortMergeJoin ExistenceJoin(exists#1) (29) - : : :- * SortMergeJoin ExistenceJoin(exists#2) (21) - : : : :- * SortMergeJoin LeftSemi (13) +TakeOrderedAndProject (54) ++- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (43) + : +- Exchange (42) + : +- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (34) + : : +- Exchange (33) + : : +- * Project (32) + : : +- * Filter (31) + : : +- * SortMergeJoin ExistenceJoin(exists#1) (30) + : : :- * SortMergeJoin ExistenceJoin(exists#2) (22) + : : : :- * SortMergeJoin LeftSemi (14) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (8) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * ColumnarToRow (15) - : : : : +- Scan parquet spark_catalog.default.web_sales (14) - : : : +- ReusedExchange (16) - : : +- * Sort (28) - : : +- Exchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- * Sort (38) - : +- Exchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet spark_catalog.default.customer_address (34) - +- * Sort (47) - +- Exchange (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet spark_catalog.default.customer_demographics (43) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet spark_catalog.default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet spark_catalog.default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- * Sort (39) + : +- Exchange (38) + : +- * Filter (37) + : +- * ColumnarToRow (36) + : +- Scan parquet spark_catalog.default.customer_address (35) + +- * Sort (48) + +- Exchange (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet spark_catalog.default.customer_demographics (44) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -66,7 +67,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -81,256 +82,261 @@ Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(8) ReusedExchange [Reuses operator id: 58] +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(9) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#9] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(11) Exchange +(12) Exchange Input [1]: [ss_customer_sk#6] Arguments: hashpartitioning(ss_customer_sk#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [1]: [ss_customer_sk#6] Arguments: [ss_customer_sk#6 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(14) Scan parquet spark_catalog.default.web_sales +(15) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 8] +(16) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(16) ReusedExchange [Reuses operator id: 58] +(17) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#12] -(17) BroadcastHashJoin [codegen id : 8] +(18) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(19) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(19) Exchange +(20) Exchange Input [1]: [ws_bill_customer_sk#10] Arguments: hashpartitioning(ws_bill_customer_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) Sort [codegen id : 9] +(21) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#10] Arguments: [ws_bill_customer_sk#10 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin [codegen id : 10] +(22) SortMergeJoin [codegen id : 10] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(22) Scan parquet spark_catalog.default.catalog_sales +(23) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 12] +(24) ColumnarToRow [codegen id : 12] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(24) ReusedExchange [Reuses operator id: 58] +(25) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#15] -(25) BroadcastHashJoin [codegen id : 12] +(26) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 12] +(27) Project [codegen id : 12] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] -(27) Exchange +(28) Exchange Input [1]: [cs_ship_customer_sk#13] Arguments: hashpartitioning(cs_ship_customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) Sort [codegen id : 13] +(29) Sort [codegen id : 13] Input [1]: [cs_ship_customer_sk#13] Arguments: [cs_ship_customer_sk#13 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 14] +(30) SortMergeJoin [codegen id : 14] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(30) Filter [codegen id : 14] +(31) Filter [codegen id : 14] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 14] +(32) Project [codegen id : 14] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Exchange +(33) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(33) Sort [codegen id : 15] +(34) Sort [codegen id : 15] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(34) Scan parquet spark_catalog.default.customer_address +(35) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 16] +(36) ColumnarToRow [codegen id : 16] Input [2]: [ca_address_sk#16, ca_state#17] -(36) Filter [codegen id : 16] +(37) Filter [codegen id : 16] Input [2]: [ca_address_sk#16, ca_state#17] Condition : isnotnull(ca_address_sk#16) -(37) Exchange +(38) Exchange Input [2]: [ca_address_sk#16, ca_state#17] Arguments: hashpartitioning(ca_address_sk#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) Sort [codegen id : 17] +(39) Sort [codegen id : 17] Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 18] +(40) SortMergeJoin [codegen id : 18] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(40) Project [codegen id : 18] +(41) Project [codegen id : 18] Output [2]: [c_current_cdemo_sk#4, ca_state#17] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] -(41) Exchange +(42) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#17] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) Sort [codegen id : 19] +(43) Sort [codegen id : 19] Input [2]: [c_current_cdemo_sk#4, ca_state#17] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(43) Scan parquet spark_catalog.default.customer_demographics +(44) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 20] +(45) ColumnarToRow [codegen id : 20] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(45) Filter [codegen id : 20] +(46) Filter [codegen id : 20] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Condition : isnotnull(cd_demo_sk#18) -(46) Exchange +(47) Exchange Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: hashpartitioning(cd_demo_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) Sort [codegen id : 21] +(48) Sort [codegen id : 21] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: [cd_demo_sk#18 ASC NULLS FIRST], false, 0 -(48) SortMergeJoin [codegen id : 22] +(49) SortMergeJoin [codegen id : 22] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(49) Project [codegen id : 22] +(50) Project [codegen id : 22] Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(50) HashAggregate [codegen id : 22] +(51) HashAggregate [codegen id : 22] Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_min(cd_dep_count#21), partial_max(cd_dep_count#21), partial_avg(cd_dep_count#21), partial_min(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_avg(cd_dep_employed_count#22), partial_min(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_avg(cd_dep_college_count#23)] Aggregate Attributes [13]: [count#24, min#25, max#26, sum#27, count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36] Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -(51) Exchange +(52) Exchange Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(52) HashAggregate [codegen id : 23] +(53) HashAggregate [codegen id : 23] Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), min(cd_dep_count#21), max(cd_dep_count#21), avg(cd_dep_count#21), min(cd_dep_employed_count#22), max(cd_dep_employed_count#22), avg(cd_dep_employed_count#22), min(cd_dep_college_count#23), max(cd_dep_college_count#23), avg(cd_dep_college_count#23)] Aggregate Attributes [10]: [count(1)#50, min(cd_dep_count#21)#51, max(cd_dep_count#21)#52, avg(cd_dep_count#21)#53, min(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, avg(cd_dep_employed_count#22)#56, min(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, avg(cd_dep_college_count#23)#59] Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, count(1)#50 AS cnt1#60, min(cd_dep_count#21)#51 AS min(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, avg(cd_dep_count#21)#53 AS avg(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, min(cd_dep_employed_count#22)#54 AS min(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, avg(cd_dep_employed_count#22)#56 AS avg(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, min(cd_dep_college_count#23)#57 AS min(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, avg(cd_dep_college_count#23)#59 AS avg(cd_dep_college_count)#71, cd_dep_count#21] -(53) TakeOrderedAndProject +(54) TakeOrderedAndProject Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71, cd_dep_count#21] Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (58) -+- * Project (57) - +- * Filter (56) - +- * ColumnarToRow (55) - +- Scan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (59) ++- * Project (58) + +- * Filter (57) + +- * ColumnarToRow (56) + +- Scan parquet spark_catalog.default.date_dim (55) -(54) Scan parquet spark_catalog.default.date_dim +(55) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(56) Filter [codegen id : 1] +(57) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 2002)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#9)) -(57) Project [codegen id : 1] +(58) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(58) BroadcastExchange +(59) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 14 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index a2c6838e17484..6ce4f6e1a81f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -48,17 +48,18 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index fad531407ab0a..866cfa7b6117a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,53 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * Filter (26) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (25) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (18) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet spark_catalog.default.customer_demographics (33) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * ColumnarToRow (13) + : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- Scan parquet spark_catalog.default.catalog_sales (19) + : : +- ReusedExchange (21) + : +- BroadcastExchange (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet spark_catalog.default.customer_demographics (34) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -55,227 +56,232 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 47] +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(9) BroadcastExchange +(10) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(11) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(11) Scan parquet spark_catalog.default.web_sales +(12) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 47] +(14) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(16) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(16) BroadcastExchange +(17) BroadcastExchange Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(18) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales +(19) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(20) ReusedExchange [Reuses operator id: 47] +(21) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] -(23) BroadcastExchange +(24) BroadcastExchange Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(26) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address +(28) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 7] +(29) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#16, ca_state#17] -(29) Filter [codegen id : 7] +(30) Filter [codegen id : 7] Input [2]: [ca_address_sk#16, ca_state#17] Condition : isnotnull(ca_address_sk#16) -(30) BroadcastExchange +(31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#17] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] -(33) Scan parquet spark_catalog.default.customer_demographics +(34) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] +(35) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(35) Filter [codegen id : 8] +(36) Filter [codegen id : 8] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Condition : isnotnull(cd_demo_sk#18) -(36) BroadcastExchange +(37) BroadcastExchange Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(39) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 9] Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_min(cd_dep_count#21), partial_max(cd_dep_count#21), partial_avg(cd_dep_count#21), partial_min(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_avg(cd_dep_employed_count#22), partial_min(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_avg(cd_dep_college_count#23)] Aggregate Attributes [13]: [count#24, min#25, max#26, sum#27, count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36] Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -(40) Exchange +(41) Exchange Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 10] Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), min(cd_dep_count#21), max(cd_dep_count#21), avg(cd_dep_count#21), min(cd_dep_employed_count#22), max(cd_dep_employed_count#22), avg(cd_dep_employed_count#22), min(cd_dep_college_count#23), max(cd_dep_college_count#23), avg(cd_dep_college_count#23)] Aggregate Attributes [10]: [count(1)#50, min(cd_dep_count#21)#51, max(cd_dep_count#21)#52, avg(cd_dep_count#21)#53, min(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, avg(cd_dep_employed_count#22)#56, min(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, avg(cd_dep_college_count#23)#59] Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, count(1)#50 AS cnt1#60, min(cd_dep_count#21)#51 AS min(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, avg(cd_dep_count#21)#53 AS avg(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, min(cd_dep_employed_count#22)#54 AS min(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, avg(cd_dep_employed_count#22)#56 AS avg(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, min(cd_dep_college_count#23)#57 AS min(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, avg(cd_dep_college_count#23)#59 AS avg(cd_dep_college_count)#71, cd_dep_count#21] -(42) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71, cd_dep_count#21] Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * Project (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (48) ++- * Project (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet spark_catalog.default.date_dim (44) -(43) Scan parquet spark_catalog.default.date_dim +(44) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(45) Filter [codegen id : 1] +(46) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 2002)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#9)) -(46) Project [codegen id : 1] +(47) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(47) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 3489f51eb24fd..d85b9a98d2b7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,17 +23,18 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter 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/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 66d33e5bd5464..a7c9ffde5491b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,13 +136,13 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) +Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_color#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] +PushedFilters: [In(i_color, [blanched ,burnished ,slate ]), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -150,7 +150,7 @@ Input [2]: [i_item_id#11, i_color#12] (19) Filter [codegen id : 3] Input [2]: [i_item_id#11, i_color#12] -Condition : i_color#12 IN (slate ,blanched ,burnished ) +Condition : (i_color#12 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#11)) (20) Project [codegen id : 3] Output [1]: [i_item_id#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 4177a855c93a9..9daca6985443b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color] + Filter [i_color,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_id,i_color] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 66d33e5bd5464..a7c9ffde5491b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,13 +136,13 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) +Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_color#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] +PushedFilters: [In(i_color, [blanched ,burnished ,slate ]), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -150,7 +150,7 @@ Input [2]: [i_item_id#11, i_color#12] (19) Filter [codegen id : 3] Input [2]: [i_item_id#11, i_color#12] -Condition : i_color#12 IN (slate ,blanched ,burnished ) +Condition : (i_color#12 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#11)) (20) Project [codegen id : 3] Output [1]: [i_item_id#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 4177a855c93a9..9daca6985443b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color] + Filter [i_color,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_id,i_color] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 7c81baa9931d3..08d5d361f1aa9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -306,7 +306,7 @@ BroadcastExchange (60) Output [2]: [d_date_sk#5, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] @@ -314,13 +314,13 @@ Input [2]: [d_date_sk#5, d_date#39] (52) Filter [codegen id : 2] Input [2]: [d_date_sk#5, d_date#39] -Condition : isnotnull(d_date_sk#5) +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#39)) (53) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (54) ColumnarToRow [codegen id : 1] @@ -328,7 +328,7 @@ Input [2]: [d_date#40, d_week_seq#41] (55) Filter [codegen id : 1] Input [2]: [d_date#40, d_week_seq#41] -Condition : (isnotnull(d_week_seq#41) AND (d_week_seq#41 = Subquery scalar-subquery#42, [id=#43])) +Condition : ((isnotnull(d_week_seq#41) AND (d_week_seq#41 = Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date#40)) (56) Project [codegen id : 1] Output [1]: [d_date#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 42403c1c39ae2..0947e6de1569e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] @@ -31,7 +31,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #3 WholeStageCodegen (1) Project [d_date] - Filter [d_week_seq] + Filter [d_week_seq,d_date] Subquery #2 WholeStageCodegen (1) Project [d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index ff8c4392d26e0..fd40efb9bb5c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -306,7 +306,7 @@ BroadcastExchange (60) Output [2]: [d_date_sk#7, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] @@ -314,13 +314,13 @@ Input [2]: [d_date_sk#7, d_date#39] (52) Filter [codegen id : 2] Input [2]: [d_date_sk#7, d_date#39] -Condition : isnotnull(d_date_sk#7) +Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#39)) (53) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (54) ColumnarToRow [codegen id : 1] @@ -328,7 +328,7 @@ Input [2]: [d_date#40, d_week_seq#41] (55) Filter [codegen id : 1] Input [2]: [d_date#40, d_week_seq#41] -Condition : (isnotnull(d_week_seq#41) AND (d_week_seq#41 = Subquery scalar-subquery#42, [id=#43])) +Condition : ((isnotnull(d_week_seq#41) AND (d_week_seq#41 = Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date#40)) (56) Project [codegen id : 1] Output [1]: [d_date#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 4b99a85d0b712..95cf2d7f56c7a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] @@ -31,7 +31,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #3 WholeStageCodegen (1) Project [d_date] - Filter [d_week_seq] + Filter [d_week_seq,d_date] Subquery #2 WholeStageCodegen (1) Project [d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 61a1f4d927a4c..51d4a65fd3f8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,13 +136,13 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) +Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_category#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -150,7 +150,7 @@ Input [2]: [i_item_id#11, i_category#12] (19) Filter [codegen id : 3] Input [2]: [i_item_id#11, i_category#12] -Condition : (isnotnull(i_category#12) AND (i_category#12 = Music )) +Condition : ((isnotnull(i_category#12) AND (i_category#12 = Music )) AND isnotnull(i_item_id#11)) (20) Project [codegen id : 3] Output [1]: [i_item_id#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 754cda4695efe..65e5199b9de8e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category] + Filter [i_category,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_id,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 61a1f4d927a4c..51d4a65fd3f8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,13 +136,13 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) +Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_category#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -150,7 +150,7 @@ Input [2]: [i_item_id#11, i_category#12] (19) Filter [codegen id : 3] Input [2]: [i_item_id#11, i_category#12] -Condition : (isnotnull(i_category#12) AND (i_category#12 = Music )) +Condition : ((isnotnull(i_category#12) AND (i_category#12 = Music )) AND isnotnull(i_item_id#11)) (20) Project [codegen id : 3] Output [1]: [i_item_id#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 754cda4695efe..65e5199b9de8e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category] + Filter [i_category,i_item_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_id,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 96ca7b8cb0be3..f2e978d074582 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -1,58 +1,61 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildLeft (42) - :- BroadcastExchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * SortMergeJoin LeftAnti (29) - : : :- * SortMergeJoin LeftAnti (21) - : : : :- * SortMergeJoin LeftSemi (13) +TakeOrderedAndProject (50) ++- * HashAggregate (49) + +- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin Inner BuildLeft (45) + :- BroadcastExchange (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (33) + : : +- * SortMergeJoin LeftAnti (32) + : : :- * SortMergeJoin LeftAnti (23) + : : : :- * SortMergeJoin LeftSemi (14) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (8) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * ColumnarToRow (15) - : : : : +- Scan parquet spark_catalog.default.web_sales (14) - : : : +- ReusedExchange (16) - : : +- * Sort (28) - : : +- Exchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet spark_catalog.default.customer_address (31) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet spark_catalog.default.customer_demographics (39) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (22) + : : : +- Exchange (21) + : : : +- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet spark_catalog.default.web_sales (15) + : : : +- ReusedExchange (18) + : : +- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (26) + : : : +- * ColumnarToRow (25) + : : : +- Scan parquet spark_catalog.default.catalog_sales (24) + : : +- ReusedExchange (27) + : +- BroadcastExchange (38) + : +- * Project (37) + : +- * Filter (36) + : +- * ColumnarToRow (35) + : +- Scan parquet spark_catalog.default.customer_address (34) + +- * Filter (44) + +- * ColumnarToRow (43) + +- Scan parquet spark_catalog.default.customer_demographics (42) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -60,7 +63,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) +Condition : (((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -75,240 +78,255 @@ Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(8) ReusedExchange [Reuses operator id: 59] +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(9) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#9] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(11) Exchange +(12) Exchange Input [1]: [ss_customer_sk#6] Arguments: hashpartitioning(ss_customer_sk#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [1]: [ss_customer_sk#6] Arguments: [ss_customer_sk#6 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(14) Scan parquet spark_catalog.default.web_sales +(15) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 8] +(16) ColumnarToRow [codegen id : 8] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] + +(17) Filter [codegen id : 8] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Condition : isnotnull(ws_bill_customer_sk#10) -(16) ReusedExchange [Reuses operator id: 59] +(18) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#12] -(17) BroadcastHashJoin [codegen id : 8] +(19) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(20) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(19) Exchange +(21) Exchange Input [1]: [ws_bill_customer_sk#10] Arguments: hashpartitioning(ws_bill_customer_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) Sort [codegen id : 9] +(22) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#10] Arguments: [ws_bill_customer_sk#10 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin [codegen id : 10] +(23) SortMergeJoin [codegen id : 10] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti Join condition: None -(22) Scan parquet spark_catalog.default.catalog_sales +(24) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 12] +(25) ColumnarToRow [codegen id : 12] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(24) ReusedExchange [Reuses operator id: 59] +(26) Filter [codegen id : 12] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Condition : isnotnull(cs_ship_customer_sk#13) + +(27) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#15] -(25) BroadcastHashJoin [codegen id : 12] +(28) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 12] +(29) Project [codegen id : 12] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] -(27) Exchange +(30) Exchange Input [1]: [cs_ship_customer_sk#13] Arguments: hashpartitioning(cs_ship_customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) Sort [codegen id : 13] +(31) Sort [codegen id : 13] Input [1]: [cs_ship_customer_sk#13] Arguments: [cs_ship_customer_sk#13 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 15] +(32) SortMergeJoin [codegen id : 15] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#13] Join type: LeftAnti Join condition: None -(30) Project [codegen id : 15] +(33) Project [codegen id : 15] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(31) Scan parquet spark_catalog.default.customer_address +(34) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 14] +(35) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#16, ca_state#17] -(33) Filter [codegen id : 14] +(36) Filter [codegen id : 14] Input [2]: [ca_address_sk#16, ca_state#17] Condition : (ca_state#17 IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) -(34) Project [codegen id : 14] +(37) Project [codegen id : 14] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(35) BroadcastExchange +(38) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 15] +(39) BroadcastHashJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(37) Project [codegen id : 15] +(40) Project [codegen id : 15] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] -(38) BroadcastExchange +(41) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) Scan parquet spark_catalog.default.customer_demographics +(42) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(40) ColumnarToRow +(43) ColumnarToRow Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -(41) Filter +(44) Filter Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Condition : isnotnull(cd_demo_sk#18) -(42) BroadcastHashJoin [codegen id : 16] +(45) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(43) Project [codegen id : 16] +(46) Project [codegen id : 16] Output [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -(44) HashAggregate [codegen id : 16] +(47) HashAggregate [codegen id : 16] Input [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#24] Results [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -(45) Exchange +(48) Exchange Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(46) HashAggregate [codegen id : 17] +(49) HashAggregate [codegen id : 17] Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#26] Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] -(47) TakeOrderedAndProject +(50) TakeOrderedAndProject Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#4, [id=#5] -ObjectHashAggregate (54) -+- Exchange (53) - +- ObjectHashAggregate (52) - +- * Project (51) - +- * Filter (50) - +- * ColumnarToRow (49) - +- Scan parquet spark_catalog.default.customer_address (48) +ObjectHashAggregate (57) ++- Exchange (56) + +- ObjectHashAggregate (55) + +- * Project (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet spark_catalog.default.customer_address (51) -(48) Scan parquet spark_catalog.default.customer_address +(51) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [2]: [ca_address_sk#16, ca_state#17] -(50) Filter [codegen id : 1] +(53) Filter [codegen id : 1] Input [2]: [ca_address_sk#16, ca_state#17] Condition : (ca_state#17 IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) -(51) Project [codegen id : 1] +(54) Project [codegen id : 1] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(52) ObjectHashAggregate +(55) ObjectHashAggregate Input [1]: [ca_address_sk#16] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#16, 42), 55556, 899992, 0, 0)] Aggregate Attributes [1]: [buf#30] Results [1]: [buf#31] -(53) Exchange +(56) Exchange Input [1]: [buf#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(54) ObjectHashAggregate +(57) ObjectHashAggregate Input [1]: [buf#31] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#16, 42), 55556, 899992, 0, 0)] @@ -316,37 +334,37 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#16, 42), 5555 Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#16, 42), 55556, 899992, 0, 0)#32 AS bloomFilter#33] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet spark_catalog.default.date_dim (55) +BroadcastExchange (62) ++- * Project (61) + +- * Filter (60) + +- * ColumnarToRow (59) + +- Scan parquet spark_catalog.default.date_dim (58) -(55) Scan parquet spark_catalog.default.date_dim +(58) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#34, d_moy#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#34, d_moy#35] -(57) Filter [codegen id : 1] +(60) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#34, d_moy#35] Condition : (((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 >= 4)) AND (d_moy#35 <= 6)) AND isnotnull(d_date_sk#9)) -(58) Project [codegen id : 1] +(61) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#34, d_moy#35] -(59) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 14 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:4 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index 7635aa1c6c3de..a066f9555cc33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 55556, 899992, 0, 0),bloomFilter,buf] Exchange #4 @@ -48,17 +48,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -69,10 +70,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -83,10 +85,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (12) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index a1df34555015c..7a12ef2307550 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,53 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin LeftAnti BuildRight (27) + : : :- * BroadcastHashJoin LeftAnti BuildRight (19) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * Project (29) - : +- * Filter (28) - : +- * ColumnarToRow (27) - : +- Scan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet spark_catalog.default.customer_demographics (33) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (18) + : : : +- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Filter (22) + : : : +- * ColumnarToRow (21) + : : : +- Scan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet spark_catalog.default.customer_demographics (36) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -55,227 +58,242 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 47] +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(7) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(9) BroadcastExchange +(10) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(11) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi Join condition: None -(11) Scan parquet spark_catalog.default.web_sales +(12) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(14) Filter [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Condition : isnotnull(ws_bill_customer_sk#8) -(13) ReusedExchange [Reuses operator id: 47] +(15) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(17) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#8] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] -(16) BroadcastExchange +(18) BroadcastExchange Input [1]: [ws_bill_customer_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#8] Join type: LeftAnti Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales +(20) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] + +(22) Filter [codegen id : 6] Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] +Condition : isnotnull(cs_ship_customer_sk#11) -(20) ReusedExchange [Reuses operator id: 47] +(23) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#13] -(21) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(25) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#11] Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] -(23) BroadcastExchange +(26) BroadcastExchange Input [1]: [cs_ship_customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#11] Join type: LeftAnti Join condition: None -(25) Project [codegen id : 9] +(28) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(26) Scan parquet spark_catalog.default.customer_address +(29) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#14, ca_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#14, ca_state#15] -(28) Filter [codegen id : 7] +(31) Filter [codegen id : 7] Input [2]: [ca_address_sk#14, ca_state#15] Condition : (ca_state#15 IN (KY,GA,NM) AND isnotnull(ca_address_sk#14)) -(29) Project [codegen id : 7] +(32) Project [codegen id : 7] Output [1]: [ca_address_sk#14] Input [2]: [ca_address_sk#14, ca_state#15] -(30) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#14] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#14] -(33) Scan parquet spark_catalog.default.customer_demographics +(36) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] +(37) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -(35) Filter [codegen id : 8] +(38) Filter [codegen id : 8] Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Condition : isnotnull(cd_demo_sk#16) -(36) BroadcastExchange +(39) BroadcastExchange Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#16] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [5]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -(39) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 9] Input [5]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Keys [5]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#22] Results [6]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, count#23] -(40) Exchange +(43) Exchange Input [6]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, count#23] Arguments: hashpartitioning(cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 10] Input [6]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, count#23] Keys [5]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#24] Results [8]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, count(1)#24 AS cnt1#25, cd_purchase_estimate#20, count(1)#24 AS cnt2#26, cd_credit_rating#21, count(1)#24 AS cnt3#27] -(42) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [8]: [cd_gender#17, cd_marital_status#18, cd_education_status#19, cnt1#25, cd_purchase_estimate#20, cnt2#26, cd_credit_rating#21, cnt3#27] Arguments: 100, [cd_gender#17 ASC NULLS FIRST, cd_marital_status#18 ASC NULLS FIRST, cd_education_status#19 ASC NULLS FIRST, cd_purchase_estimate#20 ASC NULLS FIRST, cd_credit_rating#21 ASC NULLS FIRST], [cd_gender#17, cd_marital_status#18, cd_education_status#19, cnt1#25, cd_purchase_estimate#20, cnt2#26, cd_credit_rating#21, cnt3#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (47) -+- * Project (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (50) ++- * Project (49) + +- * Filter (48) + +- * ColumnarToRow (47) + +- Scan parquet spark_catalog.default.date_dim (46) -(43) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#28, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#28, d_moy#29] -(45) Filter [codegen id : 1] +(48) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#28, d_moy#29] Condition : (((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 >= 4)) AND (d_moy#29 <= 6)) AND isnotnull(d_date_sk#7)) -(46) Project [codegen id : 1] +(49) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#28, d_moy#29] -(47) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index b517903cb847c..f67b6440a63e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -22,17 +22,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -40,10 +41,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -51,10 +53,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index d64f560f144e0..4cf423817ee2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -1,47 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * Sort (40) - +- Exchange (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Expand (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) +TakeOrderedAndProject (44) ++- * Project (43) + +- Window (42) + +- * Sort (41) + +- Exchange (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) :- * Project (6) : +- * BroadcastHashJoin Inner BuildRight (5) : :- * Filter (3) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- ReusedExchange (4) - +- BroadcastExchange (32) - +- * BroadcastHashJoin LeftSemi BuildRight (31) + +- BroadcastExchange (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) :- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet spark_catalog.default.store (7) - +- BroadcastExchange (30) - +- * Project (29) - +- * Filter (28) - +- Window (27) - +- WindowGroupLimit (26) - +- * Sort (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (13) - +- BroadcastExchange (19) - +- * Filter (18) - +- * ColumnarToRow (17) - +- Scan parquet spark_catalog.default.store (16) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * Filter (27) + +- WindowGroupLimit (26) + +- * Sort (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (13) + +- BroadcastExchange (19) + +- * Filter (18) + +- * ColumnarToRow (17) + +- Scan parquet spark_catalog.default.store (16) (1) Scan parquet spark_catalog.default.store_sales @@ -52,23 +53,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 48] +(4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 9] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -76,15 +77,15 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 7] +(8) ColumnarToRow [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(9) Filter [codegen id : 7] +(9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] @@ -101,7 +102,7 @@ Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) -(13) ReusedExchange [Reuses operator id: 48] +(13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#12] (14) BroadcastHashJoin [codegen id : 4] @@ -168,113 +169,117 @@ Arguments: [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0 Input [3]: [s_state#14, _w0#18, s_state#14] Arguments: [s_state#14], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final -(27) Window +(27) Filter [codegen id : 6] +Input [3]: [s_state#14, _w0#18, s_state#14] +Condition : isnotnull(s_state#14) + +(28) Window Input [3]: [s_state#14, _w0#18, s_state#14] Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST] -(28) Filter [codegen id : 6] +(29) Filter [codegen id : 7] Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] Condition : (ranking#19 <= 5) -(29) Project [codegen id : 6] +(30) Project [codegen id : 7] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#8] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(32) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(35) Project [codegen id : 9] Output [3]: [ss_net_profit#2, s_state#8, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(35) Expand [codegen id : 8] +(36) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#8, s_county#7] Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] -(36) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 9] Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#23] Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(37) Exchange +(38) Exchange Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 10] Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] -(39) Exchange +(40) Exchange Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] Arguments: hashpartitioning(_w1#29, _w2#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 10] +(41) Sort [codegen id : 11] Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] Arguments: [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST], false, 0 -(41) Window +(42) Window Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] Arguments: [rank(_w0#28) windowspecdefinition(_w1#29, _w2#30, _w0#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#31], [_w1#29, _w2#30], [_w0#28 DESC NULLS LAST] -(42) Project [codegen id : 11] +(43) Project [codegen id : 12] Output [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] Input [8]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30, rank_within_parent#31] -(43) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * Project (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (49) ++- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet spark_catalog.default.date_dim (45) -(44) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#32] -(46) Filter [codegen id : 1] +(47) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#32] Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= 1200)) AND (d_month_seq#32 <= 1211)) AND isnotnull(d_date_sk#5)) -(47) Project [codegen id : 1] +(48) Project [codegen id : 1] Output [1]: [d_date_sk#5] Input [2]: [d_date_sk#5, d_month_seq#32] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index f98948a8cf59a..6821b542e8dce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (12) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) + WholeStageCodegen (11) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (9) + WholeStageCodegen (10) HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] InputAdapter Exchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) + WholeStageCodegen (9) HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] @@ -34,42 +34,45 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (7) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [s_store_sk] - ColumnarToRow + WholeStageCodegen (6) + Filter [s_state] + InputAdapter + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + Exchange [s_state] #6 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Scan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index dade1b4f55c5f..220fcb317b9bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,47 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * Sort (40) - +- Exchange (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Expand (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) +TakeOrderedAndProject (44) ++- * Project (43) + +- Window (42) + +- * Sort (41) + +- Exchange (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) :- * Project (6) : +- * BroadcastHashJoin Inner BuildRight (5) : :- * Filter (3) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- ReusedExchange (4) - +- BroadcastExchange (32) - +- * BroadcastHashJoin LeftSemi BuildRight (31) + +- BroadcastExchange (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) :- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet spark_catalog.default.store (7) - +- BroadcastExchange (30) - +- * Project (29) - +- * Filter (28) - +- Window (27) - +- WindowGroupLimit (26) - +- * Sort (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (16) - : +- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet spark_catalog.default.store (13) - +- ReusedExchange (19) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * Filter (27) + +- WindowGroupLimit (26) + +- * Sort (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet spark_catalog.default.store_sales (10) + : +- BroadcastExchange (16) + : +- * Filter (15) + : +- * ColumnarToRow (14) + : +- Scan parquet spark_catalog.default.store (13) + +- ReusedExchange (19) (1) Scan parquet spark_catalog.default.store_sales @@ -52,23 +53,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 48] +(4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 9] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -76,15 +77,15 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 7] +(8) ColumnarToRow [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(9) Filter [codegen id : 7] +(9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] @@ -129,7 +130,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#13] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#13] -(19) ReusedExchange [Reuses operator id: 48] +(19) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#14] (20) BroadcastHashJoin [codegen id : 4] @@ -168,113 +169,117 @@ Arguments: [s_state#13 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0 Input [3]: [s_state#13, _w0#18, s_state#13] Arguments: [s_state#13], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final -(27) Window +(27) Filter [codegen id : 6] +Input [3]: [s_state#13, _w0#18, s_state#13] +Condition : isnotnull(s_state#13) + +(28) Window Input [3]: [s_state#13, _w0#18, s_state#13] Arguments: [rank(_w0#18) windowspecdefinition(s_state#13, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#13], [_w0#18 DESC NULLS LAST] -(28) Filter [codegen id : 6] +(29) Filter [codegen id : 7] Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19] Condition : (ranking#19 <= 5) -(29) Project [codegen id : 6] +(30) Project [codegen id : 7] Output [1]: [s_state#13] Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#13] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#8] Right keys [1]: [s_state#13] Join type: LeftSemi Join condition: None -(32) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(35) Project [codegen id : 9] Output [3]: [ss_net_profit#2, s_state#8, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(35) Expand [codegen id : 8] +(36) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#8, s_county#7] Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] -(36) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 9] Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22] Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#23] Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(37) Exchange +(38) Exchange Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 10] Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] -(39) Exchange +(40) Exchange Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] Arguments: hashpartitioning(_w1#29, _w2#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 10] +(41) Sort [codegen id : 11] Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] Arguments: [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST], false, 0 -(41) Window +(42) Window Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] Arguments: [rank(_w0#28) windowspecdefinition(_w1#29, _w2#30, _w0#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#31], [_w1#29, _w2#30], [_w0#28 DESC NULLS LAST] -(42) Project [codegen id : 11] +(43) Project [codegen id : 12] Output [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] Input [8]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30, rank_within_parent#31] -(43) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * Project (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (49) ++- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet spark_catalog.default.date_dim (45) -(44) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#32] -(46) Filter [codegen id : 1] +(47) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#32] Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= 1200)) AND (d_month_seq#32 <= 1211)) AND isnotnull(d_date_sk#5)) -(47) Project [codegen id : 1] +(48) Project [codegen id : 1] Output [1]: [d_date_sk#5] Input [2]: [d_date_sk#5, d_month_seq#32] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index e61fb9ef8d78a..288863f322b9f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,17 +1,17 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (12) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) + WholeStageCodegen (11) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (9) + WholeStageCodegen (10) HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] InputAdapter Exchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) + WholeStageCodegen (9) HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] @@ -34,42 +34,45 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (7) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Filter [s_store_sk] + WholeStageCodegen (6) + Filter [s_state] + InputAdapter + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + Exchange [s_state] #6 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 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..afd1b852e7984 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] 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..c235c1651e91f 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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index 6b9707e7e96ed..cfe301a17a036 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -62,7 +62,7 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) -(4) ReusedExchange [Reuses operator id: 62] +(4) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#5] (5) BroadcastHashJoin [codegen id : 5] @@ -136,7 +136,7 @@ Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] Condition : isnotnull(cr_item_sk#13) -(19) ReusedExchange [Reuses operator id: 62] +(19) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#16] (20) BroadcastHashJoin [codegen id : 10] @@ -209,7 +209,7 @@ Input [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] Input [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] Condition : isnotnull(wr_item_sk#24) -(34) ReusedExchange [Reuses operator id: 62] +(34) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#27] (35) BroadcastHashJoin [codegen id : 16] @@ -274,29 +274,30 @@ Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [i ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * Project (61) - +- * BroadcastHashJoin LeftSemi BuildRight (60) +BroadcastExchange (63) ++- * Project (62) + +- * BroadcastHashJoin LeftSemi BuildRight (61) :- * Filter (49) : +- * ColumnarToRow (48) : +- Scan parquet spark_catalog.default.date_dim (47) - +- BroadcastExchange (59) - +- * Project (58) - +- * BroadcastHashJoin LeftSemi BuildRight (57) - :- * ColumnarToRow (51) - : +- Scan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (56) - +- * Project (55) - +- * Filter (54) - +- * ColumnarToRow (53) - +- Scan parquet spark_catalog.default.date_dim (52) + +- BroadcastExchange (60) + +- * Project (59) + +- * BroadcastHashJoin LeftSemi BuildRight (58) + :- * Filter (52) + : +- * ColumnarToRow (51) + : +- Scan parquet spark_catalog.default.date_dim (50) + +- BroadcastExchange (57) + +- * Project (56) + +- * Filter (55) + +- * ColumnarToRow (54) + +- Scan parquet spark_catalog.default.date_dim (53) (47) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (48) ColumnarToRow [codegen id : 3] @@ -304,63 +305,69 @@ Input [2]: [d_date_sk#5, d_date#39] (49) Filter [codegen id : 3] Input [2]: [d_date_sk#5, d_date#39] -Condition : isnotnull(d_date_sk#5) +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#39)) (50) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] Input [2]: [d_date#40, d_week_seq#41] -(52) Scan parquet spark_catalog.default.date_dim +(52) Filter [codegen id : 2] +Input [2]: [d_date#40, d_week_seq#41] +Condition : (isnotnull(d_week_seq#41) AND isnotnull(d_date#40)) + +(53) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#42, d_week_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [2]: [d_date#42, d_week_seq#43] -(54) Filter [codegen id : 1] +(55) Filter [codegen id : 1] Input [2]: [d_date#42, d_week_seq#43] -Condition : cast(d_date#42 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Condition : (cast(d_date#42 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#43)) -(55) Project [codegen id : 1] +(56) Project [codegen id : 1] Output [1]: [d_week_seq#43] Input [2]: [d_date#42, d_week_seq#43] -(56) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_week_seq#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(57) BroadcastHashJoin [codegen id : 2] +(58) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_week_seq#41] Right keys [1]: [d_week_seq#43] Join type: LeftSemi Join condition: None -(58) Project [codegen id : 2] +(59) Project [codegen id : 2] Output [1]: [d_date#40] Input [2]: [d_date#40, d_week_seq#41] -(59) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date#40] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 3] +(61) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#39] Right keys [1]: [d_date#40] Join type: LeftSemi Join condition: None -(61) Project [codegen id : 3] +(62) Project [codegen id : 3] Output [1]: [d_date_sk#5] Input [2]: [d_date_sk#5, d_date#39] -(62) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#5] 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/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 0026109bc256b..e141529e772f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] @@ -31,14 +31,15 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Filter [d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_date] + Filter [d_date,d_week_seq] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 949089c0de0a1..11836de73a5d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -90,7 +90,7 @@ Join condition: None Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 62] +(10) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#7] (11) BroadcastHashJoin [codegen id : 5] @@ -149,7 +149,7 @@ Join condition: None Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#17] Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#16, i_item_id#17] -(22) ReusedExchange [Reuses operator id: 62] +(22) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 10] @@ -222,7 +222,7 @@ Join condition: None Output [3]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#28] Input [5]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26, i_item_sk#27, i_item_id#28] -(37) ReusedExchange [Reuses operator id: 62] +(37) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#29] (38) BroadcastHashJoin [codegen id : 16] @@ -274,29 +274,30 @@ Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [i ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * Project (61) - +- * BroadcastHashJoin LeftSemi BuildRight (60) +BroadcastExchange (63) ++- * Project (62) + +- * BroadcastHashJoin LeftSemi BuildRight (61) :- * Filter (49) : +- * ColumnarToRow (48) : +- Scan parquet spark_catalog.default.date_dim (47) - +- BroadcastExchange (59) - +- * Project (58) - +- * BroadcastHashJoin LeftSemi BuildRight (57) - :- * ColumnarToRow (51) - : +- Scan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (56) - +- * Project (55) - +- * Filter (54) - +- * ColumnarToRow (53) - +- Scan parquet spark_catalog.default.date_dim (52) + +- BroadcastExchange (60) + +- * Project (59) + +- * BroadcastHashJoin LeftSemi BuildRight (58) + :- * Filter (52) + : +- * ColumnarToRow (51) + : +- Scan parquet spark_catalog.default.date_dim (50) + +- BroadcastExchange (57) + +- * Project (56) + +- * Filter (55) + +- * ColumnarToRow (54) + +- Scan parquet spark_catalog.default.date_dim (53) (47) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#7, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] ReadSchema: struct (48) ColumnarToRow [codegen id : 3] @@ -304,63 +305,69 @@ Input [2]: [d_date_sk#7, d_date#39] (49) Filter [codegen id : 3] Input [2]: [d_date_sk#7, d_date#39] -Condition : isnotnull(d_date_sk#7) +Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#39)) (50) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] Input [2]: [d_date#40, d_week_seq#41] -(52) Scan parquet spark_catalog.default.date_dim +(52) Filter [codegen id : 2] +Input [2]: [d_date#40, d_week_seq#41] +Condition : (isnotnull(d_week_seq#41) AND isnotnull(d_date#40)) + +(53) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#42, d_week_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [2]: [d_date#42, d_week_seq#43] -(54) Filter [codegen id : 1] +(55) Filter [codegen id : 1] Input [2]: [d_date#42, d_week_seq#43] -Condition : cast(d_date#42 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Condition : (cast(d_date#42 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#43)) -(55) Project [codegen id : 1] +(56) Project [codegen id : 1] Output [1]: [d_week_seq#43] Input [2]: [d_date#42, d_week_seq#43] -(56) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_week_seq#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(57) BroadcastHashJoin [codegen id : 2] +(58) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_week_seq#41] Right keys [1]: [d_week_seq#43] Join type: LeftSemi Join condition: None -(58) Project [codegen id : 2] +(59) Project [codegen id : 2] Output [1]: [d_date#40] Input [2]: [d_date#40, d_week_seq#41] -(59) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date#40] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 3] +(61) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#39] Right keys [1]: [d_date#40] Join type: LeftSemi Join condition: None -(61) Project [codegen id : 3] +(62) Project [codegen id : 3] Output [1]: [d_date_sk#7] Input [2]: [d_date_sk#7, d_date#39] -(62) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#7] 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/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index f2e0a901c58c1..05a4dd73a2783 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] + Filter [d_date_sk,d_date] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] @@ -31,14 +31,15 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Filter [d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_date] + Filter [d_date,d_week_seq] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date,d_week_seq] 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-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index ff096bf4509ae..d3d332be3db5b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -1,56 +1,58 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * SortMergeJoin LeftAnti (21) + : : : :- * Project (14) + : : : : +- * SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet spark_catalog.default.web_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet spark_catalog.default.web_returns (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet spark_catalog.default.customer_address (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.web_site (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.date_dim (34) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet spark_catalog.default.web_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet spark_catalog.default.web_returns (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet spark_catalog.default.customer_address (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet spark_catalog.default.web_site (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet spark_catalog.default.date_dim (36) (1) Scan parquet spark_catalog.default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -58,7 +60,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : (((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(ws_ship_date_sk#1, 42))) +Condition : (((((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(ws_ship_date_sk#1, 42))) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -76,182 +78,192 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#15, ws_order_number#16, ws_sold_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#15, ws_order_number#16, ws_sold_date_sk#17] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [ws_warehouse_sk#15, ws_order_number#16, ws_sold_date_sk#17] +Condition : (isnotnull(ws_order_number#16) AND isnotnull(ws_warehouse_sk#15)) + +(10) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#15, ws_order_number#16] Input [3]: [ws_warehouse_sk#15, ws_order_number#16, ws_sold_date_sk#17] -(10) Exchange +(11) Exchange Input [2]: [ws_warehouse_sk#15, ws_order_number#16] Arguments: hashpartitioning(ws_order_number#16, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#15, ws_order_number#16] Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(13) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#16] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#15) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(14) Scan parquet spark_catalog.default.web_returns +(15) Scan parquet spark_catalog.default.web_returns Output [2]: [wr_order_number#18, wr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 6] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [wr_order_number#18, wr_returned_date_sk#19] + +(17) Filter [codegen id : 6] Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Condition : isnotnull(wr_order_number#18) -(16) Project [codegen id : 6] +(18) Project [codegen id : 6] Output [1]: [wr_order_number#18] Input [2]: [wr_order_number#18, wr_returned_date_sk#19] -(17) Exchange +(19) Exchange Input [1]: [wr_order_number#18] Arguments: hashpartitioning(wr_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) Sort [codegen id : 7] +(20) Sort [codegen id : 7] Input [1]: [wr_order_number#18] Arguments: [wr_order_number#18 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(21) SortMergeJoin [codegen id : 11] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#18] Join type: LeftAnti Join condition: None -(20) Scan parquet spark_catalog.default.customer_address +(22) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 8] Input [2]: [ca_address_sk#20, ca_state#21] -(22) Filter [codegen id : 8] +(24) Filter [codegen id : 8] Input [2]: [ca_address_sk#20, ca_state#21] Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) -(23) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(28) Project [codegen id : 11] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] -(27) Scan parquet spark_catalog.default.web_site +(29) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#22, web_company_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 9] Input [2]: [web_site_sk#22, web_company_name#23] -(29) Filter [codegen id : 9] +(31) Filter [codegen id : 9] Input [2]: [web_site_sk#22, web_company_name#23] Condition : ((isnotnull(web_company_name#23) AND (web_company_name#23 = pri )) AND isnotnull(web_site_sk#22)) -(30) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [1]: [web_site_sk#22] Input [2]: [web_site_sk#22, web_company_name#23] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [web_site_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#22] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#22] -(34) Scan parquet spark_catalog.default.date_dim +(36) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [2]: [d_date_sk#24, d_date#25] -(36) Filter [codegen id : 10] +(38) Filter [codegen id : 10] Input [2]: [d_date_sk#24, d_date#25] Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 1999-02-01)) AND (d_date#25 <= 1999-04-02)) AND isnotnull(d_date_sk#24)) -(37) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#25] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#24] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#30] Results [3]: [sum#28, sum#29, count#31] -(44) Exchange +(46) Exchange Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] @@ -261,45 +273,45 @@ Results [3]: [count(ws_order_number#5)#30 AS order count #32, MakeDecimal(sum(Un ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -ObjectHashAggregate (52) -+- Exchange (51) - +- ObjectHashAggregate (50) - +- * Project (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet spark_catalog.default.customer_address (46) +ObjectHashAggregate (54) ++- Exchange (53) + +- ObjectHashAggregate (52) + +- * Project (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet spark_catalog.default.customer_address (48) -(46) Scan parquet spark_catalog.default.customer_address +(48) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [2]: [ca_address_sk#20, ca_state#21] -(48) Filter [codegen id : 1] +(50) Filter [codegen id : 1] Input [2]: [ca_address_sk#20, ca_state#21] Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) -(49) Project [codegen id : 1] +(51) Project [codegen id : 1] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(50) ObjectHashAggregate +(52) ObjectHashAggregate Input [1]: [ca_address_sk#20] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 17961, 333176, 0, 0)] Aggregate Attributes [1]: [buf#35] Results [1]: [buf#36] -(51) Exchange +(53) Exchange Input [1]: [buf#36] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(52) ObjectHashAggregate +(54) ObjectHashAggregate Input [1]: [buf#36] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 17961, 333176, 0, 0)] @@ -307,45 +319,45 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 1796 Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#20, 42), 17961, 333176, 0, 0)#37 AS bloomFilter#38] Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -ObjectHashAggregate (59) -+- Exchange (58) - +- ObjectHashAggregate (57) - +- * Project (56) - +- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet spark_catalog.default.web_site (53) +ObjectHashAggregate (61) ++- Exchange (60) + +- ObjectHashAggregate (59) + +- * Project (58) + +- * Filter (57) + +- * ColumnarToRow (56) + +- Scan parquet spark_catalog.default.web_site (55) -(53) Scan parquet spark_catalog.default.web_site +(55) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#22, web_company_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [2]: [web_site_sk#22, web_company_name#23] -(55) Filter [codegen id : 1] +(57) Filter [codegen id : 1] Input [2]: [web_site_sk#22, web_company_name#23] Condition : ((isnotnull(web_company_name#23) AND (web_company_name#23 = pri )) AND isnotnull(web_site_sk#22)) -(56) Project [codegen id : 1] +(58) Project [codegen id : 1] Output [1]: [web_site_sk#22] Input [2]: [web_site_sk#22, web_company_name#23] -(57) ObjectHashAggregate +(59) ObjectHashAggregate Input [1]: [web_site_sk#22] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(web_site_sk#22, 42), 4, 144, 0, 0)] Aggregate Attributes [1]: [buf#39] Results [1]: [buf#40] -(58) Exchange +(60) Exchange Input [1]: [buf#40] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(59) ObjectHashAggregate +(61) ObjectHashAggregate Input [1]: [buf#40] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(web_site_sk#22, 42), 4, 144, 0, 0)] @@ -353,45 +365,45 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(web_site_sk#22, 42), 4, 144 Results [1]: [bloom_filter_agg(xxhash64(web_site_sk#22, 42), 4, 144, 0, 0)#41 AS bloomFilter#42] Subquery:3 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -ObjectHashAggregate (66) -+- Exchange (65) - +- ObjectHashAggregate (64) - +- * Project (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet spark_catalog.default.date_dim (60) +ObjectHashAggregate (68) ++- Exchange (67) + +- ObjectHashAggregate (66) + +- * Project (65) + +- * Filter (64) + +- * ColumnarToRow (63) + +- Scan parquet spark_catalog.default.date_dim (62) -(60) Scan parquet spark_catalog.default.date_dim +(62) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_date#25] -(62) Filter [codegen id : 1] +(64) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_date#25] Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 1999-02-01)) AND (d_date#25 <= 1999-04-02)) AND isnotnull(d_date_sk#24)) -(63) Project [codegen id : 1] +(65) Project [codegen id : 1] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#25] -(64) ObjectHashAggregate +(66) ObjectHashAggregate Input [1]: [d_date_sk#24] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(d_date_sk#24, 42), 73049, 1141755, 0, 0)] Aggregate Attributes [1]: [buf#43] Results [1]: [buf#44] -(65) Exchange +(67) Exchange Input [1]: [buf#44] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(66) ObjectHashAggregate +(68) ObjectHashAggregate Input [1]: [buf#44] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(d_date_sk#24, 42), 73049, 1141755, 0, 0)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 230b08abe0a54..5bed7df19ab42 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 17961, 333176, 0, 0),bloomFilter,buf] Exchange #3 @@ -65,9 +65,10 @@ WholeStageCodegen (12) Exchange [ws_order_number] #6 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) Sort [wr_order_number] @@ -75,9 +76,10 @@ WholeStageCodegen (12) Exchange [wr_order_number] #7 WholeStageCodegen (6) Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 91592f54abf26..648e67b7c0b34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,56 +1,58 @@ == Physical Plan == -* HashAggregate (45) -+- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * SortMergeJoin LeftAnti (19) - : : : :- * Project (13) - : : : : +- * SortMergeJoin LeftSemi (12) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * SortMergeJoin LeftAnti (21) + : : : :- * Project (14) + : : : : +- * SortMergeJoin LeftSemi (13) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- * Sort (11) - : : : : +- Exchange (10) - : : : : +- * Project (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet spark_catalog.default.web_sales (7) - : : : +- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet spark_catalog.default.web_returns (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet spark_catalog.default.date_dim (20) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.web_site (34) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet spark_catalog.default.web_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet spark_catalog.default.web_returns (15) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet spark_catalog.default.date_dim (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet spark_catalog.default.web_site (36) (1) Scan parquet spark_catalog.default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -58,7 +60,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -76,182 +78,192 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -(9) Project [codegen id : 3] +(9) Filter [codegen id : 3] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) + +(10) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#9, ws_order_number#10] Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -(10) Exchange +(11) Exchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(12) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 5] +(13) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#10] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) -(13) Project [codegen id : 5] +(14) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(14) Scan parquet spark_catalog.default.web_returns +(15) Scan parquet spark_catalog.default.web_returns Output [2]: [wr_order_number#12, wr_returned_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 6] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [wr_order_number#12, wr_returned_date_sk#13] + +(17) Filter [codegen id : 6] Input [2]: [wr_order_number#12, wr_returned_date_sk#13] +Condition : isnotnull(wr_order_number#12) -(16) Project [codegen id : 6] +(18) Project [codegen id : 6] Output [1]: [wr_order_number#12] Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -(17) Exchange +(19) Exchange Input [1]: [wr_order_number#12] Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) Sort [codegen id : 7] +(20) Sort [codegen id : 7] Input [1]: [wr_order_number#12] Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 11] +(21) SortMergeJoin [codegen id : 11] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#12] Join type: LeftAnti Join condition: None -(20) Scan parquet spark_catalog.default.date_dim +(22) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 8] +(23) ColumnarToRow [codegen id : 8] Input [2]: [d_date_sk#14, d_date#15] -(22) Filter [codegen id : 8] +(24) Filter [codegen id : 8] Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) -(23) Project [codegen id : 8] +(25) Project [codegen id : 8] Output [1]: [d_date_sk#14] Input [2]: [d_date_sk#14, d_date#15] -(24) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 11] +(28) Project [codegen id : 11] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -(27) Scan parquet spark_catalog.default.customer_address +(29) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#16, ca_state#17] -(29) Filter [codegen id : 9] +(31) Filter [codegen id : 9] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_address_sk#16)) -(30) Project [codegen id : 9] +(32) Project [codegen id : 9] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(31) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 11] +(35) Project [codegen id : 11] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -(34) Scan parquet spark_catalog.default.web_site +(36) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#18, web_company_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#18, web_company_name#19] -(36) Filter [codegen id : 10] +(38) Filter [codegen id : 10] Input [2]: [web_site_sk#18, web_company_name#19] Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri )) AND isnotnull(web_site_sk#18)) -(37) Project [codegen id : 10] +(39) Project [codegen id : 10] Output [1]: [web_site_sk#18] Input [2]: [web_site_sk#18, web_company_name#19] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(42) Project [codegen id : 11] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(41) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] Results [3]: [sum#22, sum#23, count#25] -(44) Exchange +(46) Exchange Input [3]: [sum#22, sum#23, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 12] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 86981cc8083cc..9f5f02383e358 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -35,9 +35,10 @@ WholeStageCodegen (12) Exchange [ws_order_number] #3 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) Sort [wr_order_number] @@ -45,9 +46,10 @@ WholeStageCodegen (12) Exchange [wr_order_number] #4 WholeStageCodegen (6) Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index d6cf257b8b528..f3ce4a1d9c9e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -62,7 +62,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +70,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : (((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND might_contain(Subquery scalar-subquery#8, [id=#9], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#10, [id=#11], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#12, [id=#13], xxhash64(ws_ship_date_sk#1, 42))) +Condition : ((((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) AND might_contain(Subquery scalar-subquery#8, [id=#9], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#10, [id=#11], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#12, [id=#13], xxhash64(ws_ship_date_sk#1, 42))) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 8922d43c2aaa4..f55c841465bfe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (21) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 17961, 333176, 0, 0),bloomFilter,buf] Exchange #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 194134249e6d2..513f72d9b86f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -63,7 +63,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +71,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) +Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index f628b290944a1..124708bbfbc21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (21) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 72298764a9e36..b85e98570179d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -1,56 +1,59 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildLeft (40) - :- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * SortMergeJoin LeftSemi (27) - : : :- * SortMergeJoin LeftSemi (13) +TakeOrderedAndProject (48) ++- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildLeft (43) + :- BroadcastExchange (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (31) + : : +- * SortMergeJoin LeftSemi (30) + : : :- * SortMergeJoin LeftSemi (14) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : +- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (7) - : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : +- ReusedExchange (8) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- Union (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * ColumnarToRow (15) - : : : : +- Scan parquet spark_catalog.default.web_sales (14) - : : : +- ReusedExchange (16) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet spark_catalog.default.customer_address (29) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet spark_catalog.default.customer_demographics (37) + : : : +- * Sort (13) + : : : +- Exchange (12) + : : : +- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : +- ReusedExchange (9) + : : +- * Sort (29) + : : +- Exchange (28) + : : +- Union (27) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet spark_catalog.default.web_sales (15) + : : : +- ReusedExchange (18) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (23) + : : : +- * ColumnarToRow (22) + : : : +- Scan parquet spark_catalog.default.catalog_sales (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet spark_catalog.default.customer_address (32) + +- * Filter (42) + +- * ColumnarToRow (41) + +- Scan parquet spark_catalog.default.customer_demographics (40) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -58,7 +61,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) +Condition : (((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -73,228 +76,243 @@ Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(8) ReusedExchange [Reuses operator id: 57] +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(9) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#9] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(11) Exchange +(12) Exchange Input [1]: [ss_customer_sk#6] Arguments: hashpartitioning(ss_customer_sk#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [1]: [ss_customer_sk#6] Arguments: [ss_customer_sk#6 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(14) Scan parquet spark_catalog.default.web_sales +(15) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 8] +(16) ColumnarToRow [codegen id : 8] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] + +(17) Filter [codegen id : 8] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Condition : isnotnull(ws_bill_customer_sk#10) -(16) ReusedExchange [Reuses operator id: 57] +(18) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#12] -(17) BroadcastHashJoin [codegen id : 8] +(19) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(20) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#10 AS customer_sk#13] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(19) Scan parquet spark_catalog.default.catalog_sales +(21) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 10] +(22) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(21) ReusedExchange [Reuses operator id: 57] +(23) Filter [codegen id : 10] +Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Condition : isnotnull(cs_ship_customer_sk#14) + +(24) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#16] -(22) BroadcastHashJoin [codegen id : 10] +(25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#15] Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(23) Project [codegen id : 10] +(26) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#14 AS customer_sk#17] Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#16] -(24) Union +(27) Union -(25) Exchange +(28) Exchange Input [1]: [customer_sk#13] Arguments: hashpartitioning(customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(26) Sort [codegen id : 11] +(29) Sort [codegen id : 11] Input [1]: [customer_sk#13] Arguments: [customer_sk#13 ASC NULLS FIRST], false, 0 -(27) SortMergeJoin [codegen id : 13] +(30) SortMergeJoin [codegen id : 13] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#13] Join type: LeftSemi Join condition: None -(28) Project [codegen id : 13] +(31) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(29) Scan parquet spark_catalog.default.customer_address +(32) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#18, ca_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 12] +(33) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#18, ca_county#19] -(31) Filter [codegen id : 12] +(34) Filter [codegen id : 12] Input [2]: [ca_address_sk#18, ca_county#19] Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) -(32) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_county#19] -(33) BroadcastExchange +(36) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 13] +(37) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 13] +(38) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(36) BroadcastExchange +(39) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(37) Scan parquet spark_catalog.default.customer_demographics +(40) Scan parquet spark_catalog.default.customer_demographics Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(38) ColumnarToRow +(41) ColumnarToRow Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) Filter +(42) Filter Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#20) -(40) BroadcastHashJoin [codegen id : 14] +(43) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 14] +(44) Project [codegen id : 14] Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(42) HashAggregate [codegen id : 14] +(45) HashAggregate [codegen id : 14] Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(43) Exchange +(46) Exchange Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(44) HashAggregate [codegen id : 15] +(47) HashAggregate [codegen id : 15] Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] -(45) TakeOrderedAndProject +(48) TakeOrderedAndProject Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#4, [id=#5] -ObjectHashAggregate (52) -+- Exchange (51) - +- ObjectHashAggregate (50) - +- * Project (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet spark_catalog.default.customer_address (46) +ObjectHashAggregate (55) ++- Exchange (54) + +- ObjectHashAggregate (53) + +- * Project (52) + +- * Filter (51) + +- * ColumnarToRow (50) + +- Scan parquet spark_catalog.default.customer_address (49) -(46) Scan parquet spark_catalog.default.customer_address +(49) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#18, ca_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [2]: [ca_address_sk#18, ca_county#19] -(48) Filter [codegen id : 1] +(51) Filter [codegen id : 1] Input [2]: [ca_address_sk#18, ca_county#19] Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) -(49) Project [codegen id : 1] +(52) Project [codegen id : 1] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_county#19] -(50) ObjectHashAggregate +(53) ObjectHashAggregate Input [1]: [ca_address_sk#18] Keys: [] Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)] Aggregate Attributes [1]: [buf#38] Results [1]: [buf#39] -(51) Exchange +(54) Exchange Input [1]: [buf#39] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(52) ObjectHashAggregate +(55) ObjectHashAggregate Input [1]: [buf#39] Keys: [] Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)] @@ -302,37 +320,37 @@ Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555 Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)#40 AS bloomFilter#41] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * Project (56) - +- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (60) ++- * Project (59) + +- * Filter (58) + +- * ColumnarToRow (57) + +- Scan parquet spark_catalog.default.date_dim (56) -(53) Scan parquet spark_catalog.default.date_dim +(56) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(57) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#42, d_moy#43] -(55) Filter [codegen id : 1] +(58) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#42, d_moy#43] Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 7)) AND isnotnull(d_date_sk#9)) -(56) Project [codegen id : 1] +(59) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#42, d_moy#43] -(57) BroadcastExchange +(60) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:3 Hosting operator id = 14 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 49da06e14bc21..747b6ab4dd8cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 2555, 57765, 0, 0),bloomFilter,buf] Exchange #4 @@ -45,17 +45,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -67,19 +68,21 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index da00acc535c08..59c47a8b285a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,52 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : :- * BroadcastHashJoin LeftSemi BuildRight (11) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * Filter (27) - : +- * ColumnarToRow (26) - : +- Scan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (35) - +- * Filter (34) - +- * ColumnarToRow (33) - +- Scan parquet spark_catalog.default.customer_demographics (32) + : : : +- BroadcastExchange (10) + : : : +- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (7) + : : +- BroadcastExchange (25) + : : +- Union (24) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (15) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -54,219 +57,234 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 46] +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(7) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(9) BroadcastExchange +(10) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(11) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi Join condition: None -(11) Scan parquet spark_catalog.default.web_sales +(12) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(14) Filter [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Condition : isnotnull(ws_bill_customer_sk#8) -(13) ReusedExchange [Reuses operator id: 46] +(15) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(17) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#8 AS customer_sk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] -(16) Scan parquet spark_catalog.default.catalog_sales +(18) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] + +(20) Filter [codegen id : 6] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +Condition : isnotnull(cs_ship_customer_sk#12) -(18) ReusedExchange [Reuses operator id: 46] +(21) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] -(21) Union +(24) Union -(22) BroadcastExchange +(25) BroadcastExchange Input [1]: [customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#11] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) Scan parquet spark_catalog.default.customer_address +(28) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_county#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 7] +(29) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#16, ca_county#17] -(27) Filter [codegen id : 7] +(30) Filter [codegen id : 7] Input [2]: [ca_address_sk#16, ca_county#17] Condition : (ca_county#17 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#16)) -(28) Project [codegen id : 7] +(31) Project [codegen id : 7] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_county#17] -(29) BroadcastExchange +(32) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(34) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] -(32) Scan parquet spark_catalog.default.customer_demographics +(35) Scan parquet spark_catalog.default.customer_demographics Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 8] +(36) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(34) Filter [codegen id : 8] +(37) Filter [codegen id : 8] Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Condition : isnotnull(cd_demo_sk#18) -(35) BroadcastExchange +(38) BroadcastExchange Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(37) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(38) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#27] Results [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] -(39) Exchange +(42) Exchange Input [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] Keys [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#29] Results [14]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#29 AS cnt1#30, cd_purchase_estimate#22, count(1)#29 AS cnt2#31, cd_credit_rating#23, count(1)#29 AS cnt3#32, cd_dep_count#24, count(1)#29 AS cnt4#33, cd_dep_employed_count#25, count(1)#29 AS cnt5#34, cd_dep_college_count#26, count(1)#29 AS cnt6#35] -(41) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [14]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#30, cd_purchase_estimate#22, cnt2#31, cd_credit_rating#23, cnt3#32, cd_dep_count#24, cnt4#33, cd_dep_employed_count#25, cnt5#34, cd_dep_college_count#26, cnt6#35] Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#30, cd_purchase_estimate#22, cnt2#31, cd_credit_rating#23, cnt3#32, cd_dep_count#24, cnt4#33, cd_dep_employed_count#25, cnt5#34, cd_dep_college_count#26, cnt6#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * Project (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (49) ++- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet spark_catalog.default.date_dim (45) -(42) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#36, d_moy#37] -(44) Filter [codegen id : 1] +(47) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#36, d_moy#37] Condition : (((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2002)) AND (d_moy#37 >= 4)) AND (d_moy#37 <= 7)) AND isnotnull(d_date_sk#7)) -(45) Project [codegen id : 1] +(48) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#36, d_moy#37] -(46) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index c140487f66150..277b01735feff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,17 +21,18 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -40,19 +41,21 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter 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 d7f8bfd6a1277..b4dbb51115112 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 @@ -110,7 +110,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 17] @@ -118,7 +118,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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 edd3486498691..0d6f76d6e73bb 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 @@ -72,7 +72,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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 603119ba166e7..614f0a6da6f1f 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 @@ -104,7 +104,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -112,7 +112,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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 8d8dcccd5d70c..fb2e53c91965a 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 @@ -72,7 +72,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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 9527bf6a06116..5910703da16ff 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 @@ -151,7 +151,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 17] @@ -159,7 +159,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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..b93f046fa0a81 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 @@ -87,7 +87,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (17) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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 ac53b8f2312c5..9d2105438bbef 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 @@ -145,7 +145,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -153,7 +153,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) (7) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] 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..ecd7e897d97b2 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 @@ -87,7 +87,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id] + Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] 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/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index f870c282ae987..160a74a3b4816 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -1,64 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (53) -+- * HashAggregate (52) - +- Exchange (51) - +- * HashAggregate (50) - +- * Project (49) - +- * SortMergeJoin Inner (48) - :- * Sort (42) - : +- Exchange (41) - : +- * Project (40) - : +- * SortMergeJoin Inner (39) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * SortMergeJoin ExistenceJoin(exists#1) (29) - : : :- * SortMergeJoin ExistenceJoin(exists#2) (21) - : : : :- * SortMergeJoin LeftSemi (13) +TakeOrderedAndProject (54) ++- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (43) + : +- Exchange (42) + : +- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (34) + : : +- Exchange (33) + : : +- * Project (32) + : : +- * Filter (31) + : : +- * SortMergeJoin ExistenceJoin(exists#1) (30) + : : :- * SortMergeJoin ExistenceJoin(exists#2) (22) + : : : :- * SortMergeJoin LeftSemi (14) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (8) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * ColumnarToRow (15) - : : : : +- Scan parquet spark_catalog.default.web_sales (14) - : : : +- ReusedExchange (16) - : : +- * Sort (28) - : : +- Exchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * ColumnarToRow (23) - : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- * Sort (38) - : +- Exchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet spark_catalog.default.customer_address (34) - +- * Sort (47) - +- Exchange (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet spark_catalog.default.customer_demographics (43) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet spark_catalog.default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet spark_catalog.default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- * Sort (39) + : +- Exchange (38) + : +- * Filter (37) + : +- * ColumnarToRow (36) + : +- Scan parquet spark_catalog.default.customer_address (35) + +- * Sort (48) + +- Exchange (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet spark_catalog.default.customer_demographics (44) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -66,7 +67,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -81,256 +82,261 @@ Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(8) ReusedExchange [Reuses operator id: 58] +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(9) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#9] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(11) Exchange +(12) Exchange Input [1]: [ss_customer_sk#6] Arguments: hashpartitioning(ss_customer_sk#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [1]: [ss_customer_sk#6] Arguments: [ss_customer_sk#6 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(14) Scan parquet spark_catalog.default.web_sales +(15) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 8] +(16) ColumnarToRow [codegen id : 8] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(16) ReusedExchange [Reuses operator id: 58] +(17) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#12] -(17) BroadcastHashJoin [codegen id : 8] +(18) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(19) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(19) Exchange +(20) Exchange Input [1]: [ws_bill_customer_sk#10] Arguments: hashpartitioning(ws_bill_customer_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(20) Sort [codegen id : 9] +(21) Sort [codegen id : 9] Input [1]: [ws_bill_customer_sk#10] Arguments: [ws_bill_customer_sk#10 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin [codegen id : 10] +(22) SortMergeJoin [codegen id : 10] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(22) Scan parquet spark_catalog.default.catalog_sales +(23) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 12] +(24) ColumnarToRow [codegen id : 12] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(24) ReusedExchange [Reuses operator id: 58] +(25) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#15] -(25) BroadcastHashJoin [codegen id : 12] +(26) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 12] +(27) Project [codegen id : 12] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] -(27) Exchange +(28) Exchange Input [1]: [cs_ship_customer_sk#13] Arguments: hashpartitioning(cs_ship_customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(28) Sort [codegen id : 13] +(29) Sort [codegen id : 13] Input [1]: [cs_ship_customer_sk#13] Arguments: [cs_ship_customer_sk#13 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 14] +(30) SortMergeJoin [codegen id : 14] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(30) Filter [codegen id : 14] +(31) Filter [codegen id : 14] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 14] +(32) Project [codegen id : 14] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Exchange +(33) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(33) Sort [codegen id : 15] +(34) Sort [codegen id : 15] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(34) Scan parquet spark_catalog.default.customer_address +(35) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 16] +(36) ColumnarToRow [codegen id : 16] Input [2]: [ca_address_sk#16, ca_state#17] -(36) Filter [codegen id : 16] +(37) Filter [codegen id : 16] Input [2]: [ca_address_sk#16, ca_state#17] Condition : isnotnull(ca_address_sk#16) -(37) Exchange +(38) Exchange Input [2]: [ca_address_sk#16, ca_state#17] Arguments: hashpartitioning(ca_address_sk#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) Sort [codegen id : 17] +(39) Sort [codegen id : 17] Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 18] +(40) SortMergeJoin [codegen id : 18] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(40) Project [codegen id : 18] +(41) Project [codegen id : 18] Output [2]: [c_current_cdemo_sk#4, ca_state#17] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] -(41) Exchange +(42) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#17] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) Sort [codegen id : 19] +(43) Sort [codegen id : 19] Input [2]: [c_current_cdemo_sk#4, ca_state#17] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(43) Scan parquet spark_catalog.default.customer_demographics +(44) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 20] +(45) ColumnarToRow [codegen id : 20] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(45) Filter [codegen id : 20] +(46) Filter [codegen id : 20] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Condition : isnotnull(cd_demo_sk#18) -(46) Exchange +(47) Exchange Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: hashpartitioning(cd_demo_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) Sort [codegen id : 21] +(48) Sort [codegen id : 21] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: [cd_demo_sk#18 ASC NULLS FIRST], false, 0 -(48) SortMergeJoin [codegen id : 22] +(49) SortMergeJoin [codegen id : 22] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(49) Project [codegen id : 22] +(50) Project [codegen id : 22] Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(50) HashAggregate [codegen id : 22] +(51) HashAggregate [codegen id : 22] Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] Aggregate Attributes [13]: [count#24, sum#25, count#26, max#27, sum#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36] Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -(51) Exchange +(52) Exchange Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(52) HashAggregate [codegen id : 23] +(53) HashAggregate [codegen id : 23] Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#21)#51, max(cd_dep_count#21)#52, sum(cd_dep_count#21)#53, avg(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, sum(cd_dep_employed_count#22)#56, avg(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, sum(cd_dep_college_count#23)#59] Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, count(1)#50 AS cnt1#60, avg(cd_dep_count#21)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#21)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#22)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#22)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#23)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#23)#59 AS sum(cd_dep_college_count)#71] -(53) TakeOrderedAndProject +(54) TakeOrderedAndProject Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (58) -+- * Project (57) - +- * Filter (56) - +- * ColumnarToRow (55) - +- Scan parquet spark_catalog.default.date_dim (54) +BroadcastExchange (59) ++- * Project (58) + +- * Filter (57) + +- * ColumnarToRow (56) + +- Scan parquet spark_catalog.default.date_dim (55) -(54) Scan parquet spark_catalog.default.date_dim +(55) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 1] +(56) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(56) Filter [codegen id : 1] +(57) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 2002)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#9)) -(57) Project [codegen id : 1] +(58) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(58) BroadcastExchange +(59) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 14 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index 78347a99cddea..e0538113b2fdf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -48,17 +48,18 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 2171a962a3d73..af84a241d9c92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,53 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * Filter (26) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (25) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (18) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet spark_catalog.default.customer_demographics (33) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * ColumnarToRow (13) + : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * ColumnarToRow (20) + : : : +- Scan parquet spark_catalog.default.catalog_sales (19) + : : +- ReusedExchange (21) + : +- BroadcastExchange (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet spark_catalog.default.customer_demographics (34) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -55,227 +56,232 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) +Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) ReusedExchange [Reuses operator id: 47] +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#6) + +(7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(9) BroadcastExchange +(10) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(11) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(11) Scan parquet spark_catalog.default.web_sales +(12) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(13) ReusedExchange [Reuses operator id: 47] +(14) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(16) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] -(16) BroadcastExchange +(17) BroadcastExchange Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(18) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales +(19) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(20) ReusedExchange [Reuses operator id: 47] +(21) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] -(23) BroadcastExchange +(24) BroadcastExchange Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(26) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address +(28) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 7] +(29) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#16, ca_state#17] -(29) Filter [codegen id : 7] +(30) Filter [codegen id : 7] Input [2]: [ca_address_sk#16, ca_state#17] Condition : isnotnull(ca_address_sk#16) -(30) BroadcastExchange +(31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#17] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] -(33) Scan parquet spark_catalog.default.customer_demographics +(34) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] +(35) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(35) Filter [codegen id : 8] +(36) Filter [codegen id : 8] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Condition : isnotnull(cd_demo_sk#18) -(36) BroadcastExchange +(37) BroadcastExchange Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(39) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 9] Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] Aggregate Attributes [13]: [count#24, sum#25, count#26, max#27, sum#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36] Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -(40) Exchange +(41) Exchange Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 10] Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#21)#51, max(cd_dep_count#21)#52, sum(cd_dep_count#21)#53, avg(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, sum(cd_dep_employed_count#22)#56, avg(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, sum(cd_dep_college_count#23)#59] Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, count(1)#50 AS cnt1#60, avg(cd_dep_count#21)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#21)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#22)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#22)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#23)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#23)#59 AS sum(cd_dep_college_count)#71] -(42) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * Project (46) - +- * Filter (45) - +- * ColumnarToRow (44) - +- Scan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (48) ++- * Project (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet spark_catalog.default.date_dim (44) -(43) Scan parquet spark_catalog.default.date_dim +(44) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(45) Filter [codegen id : 1] +(46) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 2002)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#9)) -(46) Project [codegen id : 1] +(47) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(47) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 99c08f81a0a95..35629eada6ab1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,17 +23,18 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index ec935cc0262c0..15e58db64a798 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -1,61 +1,64 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * HashAggregate (49) - +- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * SortMergeJoin Inner (45) - :- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * SortMergeJoin Inner (36) - : :- * Sort (30) - : : +- Exchange (29) - : : +- * Project (28) - : : +- * SortMergeJoin LeftSemi (27) - : : :- * SortMergeJoin LeftSemi (13) +TakeOrderedAndProject (53) ++- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- * Project (49) + +- * SortMergeJoin Inner (48) + :- * Sort (42) + : +- Exchange (41) + : +- * Project (40) + : +- * SortMergeJoin Inner (39) + : :- * Sort (33) + : : +- Exchange (32) + : : +- * Project (31) + : : +- * SortMergeJoin LeftSemi (30) + : : :- * SortMergeJoin LeftSemi (14) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : +- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (7) - : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : +- ReusedExchange (8) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- Union (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * ColumnarToRow (15) - : : : : +- Scan parquet spark_catalog.default.web_sales (14) - : : : +- ReusedExchange (16) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- * Sort (35) - : +- Exchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet spark_catalog.default.customer_address (31) - +- * Sort (44) - +- Exchange (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet spark_catalog.default.customer_demographics (40) + : : : +- * Sort (13) + : : : +- Exchange (12) + : : : +- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : +- ReusedExchange (9) + : : +- * Sort (29) + : : +- Exchange (28) + : : +- Union (27) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet spark_catalog.default.web_sales (15) + : : : +- ReusedExchange (18) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (23) + : : : +- * ColumnarToRow (22) + : : : +- Scan parquet spark_catalog.default.catalog_sales (21) + : : +- ReusedExchange (24) + : +- * Sort (38) + : +- Exchange (37) + : +- * Filter (36) + : +- * ColumnarToRow (35) + : +- Scan parquet spark_catalog.default.customer_address (34) + +- * Sort (47) + +- Exchange (46) + +- * Filter (45) + +- * ColumnarToRow (44) + +- Scan parquet spark_catalog.default.customer_demographics (43) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -63,7 +66,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -78,240 +81,255 @@ Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(8) ReusedExchange [Reuses operator id: 55] +(8) Filter [codegen id : 4] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(9) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#7] -(9) BroadcastHashJoin [codegen id : 4] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(11) Project [codegen id : 4] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(11) Exchange +(12) Exchange Input [1]: [ss_customer_sk#4] Arguments: hashpartitioning(ss_customer_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) Sort [codegen id : 5] +(13) Sort [codegen id : 5] Input [1]: [ss_customer_sk#4] Arguments: [ss_customer_sk#4 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 6] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi Join condition: None -(14) Scan parquet spark_catalog.default.web_sales +(15) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 8] +(16) ColumnarToRow [codegen id : 8] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(17) Filter [codegen id : 8] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Condition : isnotnull(ws_bill_customer_sk#8) -(16) ReusedExchange [Reuses operator id: 55] +(18) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 8] +(19) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(20) Project [codegen id : 8] Output [1]: [ws_bill_customer_sk#8 AS customsk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] -(19) Scan parquet spark_catalog.default.catalog_sales +(21) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 10] +(22) ColumnarToRow [codegen id : 10] +Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] + +(23) Filter [codegen id : 10] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +Condition : isnotnull(cs_ship_customer_sk#12) -(21) ReusedExchange [Reuses operator id: 55] +(24) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#14] -(22) BroadcastHashJoin [codegen id : 10] +(25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(23) Project [codegen id : 10] +(26) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#12 AS customsk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] -(24) Union +(27) Union -(25) Exchange +(28) Exchange Input [1]: [customsk#11] Arguments: hashpartitioning(customsk#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(26) Sort [codegen id : 11] +(29) Sort [codegen id : 11] Input [1]: [customsk#11] Arguments: [customsk#11 ASC NULLS FIRST], false, 0 -(27) SortMergeJoin [codegen id : 12] +(30) SortMergeJoin [codegen id : 12] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#11] Join type: LeftSemi Join condition: None -(28) Project [codegen id : 12] +(31) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(29) Exchange +(32) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(30) Sort [codegen id : 13] +(33) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 -(31) Scan parquet spark_catalog.default.customer_address +(34) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 14] +(35) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#16, ca_state#17] -(33) Filter [codegen id : 14] +(36) Filter [codegen id : 14] Input [2]: [ca_address_sk#16, ca_state#17] Condition : isnotnull(ca_address_sk#16) -(34) Exchange +(37) Exchange Input [2]: [ca_address_sk#16, ca_state#17] Arguments: hashpartitioning(ca_address_sk#16, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(35) Sort [codegen id : 15] +(38) Sort [codegen id : 15] Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin [codegen id : 16] +(39) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(37) Project [codegen id : 16] +(40) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#2, ca_state#17] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16, ca_state#17] -(38) Exchange +(41) Exchange Input [2]: [c_current_cdemo_sk#2, ca_state#17] Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) Sort [codegen id : 17] +(42) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#2, ca_state#17] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 -(40) Scan parquet spark_catalog.default.customer_demographics +(43) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 18] +(44) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(42) Filter [codegen id : 18] +(45) Filter [codegen id : 18] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Condition : isnotnull(cd_demo_sk#18) -(43) Exchange +(46) Exchange Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: hashpartitioning(cd_demo_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(44) Sort [codegen id : 19] +(47) Sort [codegen id : 19] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: [cd_demo_sk#18 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 20] +(48) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(46) Project [codegen id : 20] +(49) Project [codegen id : 20] Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#2, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(47) HashAggregate [codegen id : 20] +(50) HashAggregate [codegen id : 20] Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] Aggregate Attributes [13]: [count#24, sum#25, count#26, max#27, sum#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36] Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -(48) Exchange +(51) Exchange Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(49) HashAggregate [codegen id : 21] +(52) HashAggregate [codegen id : 21] Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#21)#51, max(cd_dep_count#21)#52, sum(cd_dep_count#21)#53, avg(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, sum(cd_dep_employed_count#22)#56, avg(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, sum(cd_dep_college_count#23)#59] Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, count(1)#50 AS cnt1#60, avg(cd_dep_count#21)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#21)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#22)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#22)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#23)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#23)#59 AS sum(cd_dep_college_count)#71] -(50) TakeOrderedAndProject +(53) TakeOrderedAndProject Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (55) -+- * Project (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (58) ++- * Project (57) + +- * Filter (56) + +- * ColumnarToRow (55) + +- Scan parquet spark_catalog.default.date_dim (54) -(51) Scan parquet spark_catalog.default.date_dim +(54) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#72, d_qoy#73] -(53) Filter [codegen id : 1] +(56) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 1999)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#7)) -(54) Project [codegen id : 1] +(57) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#72, d_qoy#73] -(55) BroadcastExchange +(58) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 14 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 210659ee63176..2751f923bd2d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -44,17 +44,18 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -66,19 +67,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (8) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 6dc88cf99cac8..362cc13721c83 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,51 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : :- * BroadcastHashJoin LeftSemi BuildRight (11) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (28) - : +- * Filter (27) - : +- * ColumnarToRow (26) - : +- Scan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet spark_catalog.default.customer_demographics (31) + : : : +- BroadcastExchange (10) + : : : +- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (7) + : : +- BroadcastExchange (25) + : : +- Union (24) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (15) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (21) + : +- BroadcastExchange (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet spark_catalog.default.customer_demographics (34) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -53,215 +56,230 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) +Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) ReusedExchange [Reuses operator id: 45] +(6) Filter [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Condition : isnotnull(ss_customer_sk#4) + +(7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(9) BroadcastExchange +(10) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(11) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi Join condition: None -(11) Scan parquet spark_catalog.default.web_sales +(12) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(14) Filter [codegen id : 4] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Condition : isnotnull(ws_bill_customer_sk#8) -(13) ReusedExchange [Reuses operator id: 45] +(15) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(17) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#8 AS customsk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] -(16) Scan parquet spark_catalog.default.catalog_sales +(18) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] + +(20) Filter [codegen id : 6] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +Condition : isnotnull(cs_ship_customer_sk#12) -(18) ReusedExchange [Reuses operator id: 45] +(21) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(23) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#12 AS customsk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] -(21) Union +(24) Union -(22) BroadcastExchange +(25) BroadcastExchange Input [1]: [customsk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#11] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) Scan parquet spark_catalog.default.customer_address +(28) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 7] +(29) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#16, ca_state#17] -(27) Filter [codegen id : 7] +(30) Filter [codegen id : 7] Input [2]: [ca_address_sk#16, ca_state#17] Condition : isnotnull(ca_address_sk#16) -(28) BroadcastExchange +(31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(30) Project [codegen id : 9] +(33) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, ca_state#17] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16, ca_state#17] -(31) Scan parquet spark_catalog.default.customer_demographics +(34) Scan parquet spark_catalog.default.customer_demographics Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 8] +(35) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(33) Filter [codegen id : 8] +(36) Filter [codegen id : 8] Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Condition : isnotnull(cd_demo_sk#18) -(34) BroadcastExchange +(37) BroadcastExchange Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(35) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(39) Project [codegen id : 9] Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#2, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(37) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 9] Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] Aggregate Attributes [13]: [count#24, sum#25, count#26, max#27, sum#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36] Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -(38) Exchange +(41) Exchange Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 10] Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#21)#51, max(cd_dep_count#21)#52, sum(cd_dep_count#21)#53, avg(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, sum(cd_dep_employed_count#22)#56, avg(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, sum(cd_dep_college_count#23)#59] Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, count(1)#50 AS cnt1#60, avg(cd_dep_count#21)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#21)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#22)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#22)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#23)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#23)#59 AS sum(cd_dep_college_count)#71] -(40) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * Project (44) - +- * Filter (43) - +- * ColumnarToRow (42) - +- Scan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (48) ++- * Project (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet spark_catalog.default.date_dim (44) -(41) Scan parquet spark_catalog.default.date_dim +(44) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#72, d_qoy#73] -(43) Filter [codegen id : 1] +(46) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 1999)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#7)) -(44) Project [codegen id : 1] +(47) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#72, d_qoy#73] -(45) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index d86207af22b75..18d39e8d1cc76 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,17 +21,18 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -40,19 +41,21 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [cs_ship_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter 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 497fe172c9bf6..8ecffb517bc14 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#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(77) HashAggregate [codegen id : 48] +(78) HashAggregate [codegen id : 48] 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 [4]: [channel#37, sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(78) HashAggregate [codegen id : 48] +(79) HashAggregate [codegen id : 48] Input [4]: [channel#37, sales#145, returns#146, profit#147] Keys [1]: [channel#37] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] Results [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -(79) Exchange +(80) Exchange Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(80) HashAggregate [codegen id : 49] +(81) HashAggregate [codegen id : 49] Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Keys [1]: [channel#37] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#160, sum(returns#146)#161, sum(profit#147)#162] Results [5]: [channel#37, null AS id#163, sum(sales#145)#160 AS sum(sales)#164, sum(returns#146)#161 AS sum(returns)#165, sum(profit#147)#162 AS sum(profit)#166] -(81) ReusedExchange [Reuses operator id: 74] +(82) ReusedExchange [Reuses operator id: 75] Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(82) HashAggregate [codegen id : 73] +(83) HashAggregate [codegen id : 73] 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 [3]: [sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(83) HashAggregate [codegen id : 73] +(84) HashAggregate [codegen id : 73] Input [3]: [sales#145, returns#146, profit#147] Keys: [] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] Results [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(84) Exchange +(85) Exchange Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(85) HashAggregate [codegen id : 74] +(86) HashAggregate [codegen id : 74] Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys: [] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#179, sum(returns#146)#180, sum(profit#147)#181] Results [5]: [null AS channel#182, null AS id#183, sum(sales#145)#179 AS sum(sales)#184, sum(returns#146)#180 AS sum(returns)#185, sum(profit#147)#181 AS sum(profit)#186] -(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#187] 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#187] -(93) Filter [codegen id : 1] +(94) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_date#187] Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 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#187] -(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 04d9e50272b3c..2408f9eb3c714 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#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(74) HashAggregate [codegen id : 42] +(75) HashAggregate [codegen id : 42] 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 [4]: [channel#37, sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(75) HashAggregate [codegen id : 42] +(76) HashAggregate [codegen id : 42] Input [4]: [channel#37, sales#145, returns#146, profit#147] Keys [1]: [channel#37] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] Results [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -(76) Exchange +(77) Exchange Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(77) HashAggregate [codegen id : 43] +(78) HashAggregate [codegen id : 43] Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Keys [1]: [channel#37] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#160, sum(returns#146)#161, sum(profit#147)#162] Results [5]: [channel#37, null AS id#163, sum(sales#145)#160 AS sum(sales)#164, sum(returns#146)#161 AS sum(returns)#165, sum(profit#147)#162 AS sum(profit)#166] -(78) ReusedExchange [Reuses operator id: 71] +(79) ReusedExchange [Reuses operator id: 72] Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(79) HashAggregate [codegen id : 64] +(80) HashAggregate [codegen id : 64] 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 [3]: [sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(80) HashAggregate [codegen id : 64] +(81) HashAggregate [codegen id : 64] Input [3]: [sales#145, returns#146, profit#147] Keys: [] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] Results [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(81) Exchange +(82) Exchange Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(82) HashAggregate [codegen id : 65] +(83) HashAggregate [codegen id : 65] Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys: [] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#179, sum(returns#146)#180, sum(profit#147)#181] Results [5]: [null AS channel#182, null AS id#183, sum(sales#145)#179 AS sum(sales)#184, sum(returns#146)#180 AS sum(returns)#185, sum(profit#147)#181 AS sum(profit)#186] -(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#187] 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#187] -(90) Filter [codegen id : 1] +(91) Filter [codegen id : 1] Input [2]: [d_date_sk#22, d_date#187] Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 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#187] -(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 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index ec13bd91484e4..1546b2928c357 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -1,60 +1,61 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * Project (55) - +- Window (54) - +- * Sort (53) - +- Exchange (52) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) +TakeOrderedAndProject (57) ++- * Project (56) + +- Window (55) + +- * Sort (54) + +- Exchange (53) + +- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- Union (49) + :- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) : :- * Project (6) : : +- * BroadcastHashJoin Inner BuildRight (5) : : :- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) - : +- BroadcastExchange (32) - : +- * BroadcastHashJoin LeftSemi BuildRight (31) + : +- BroadcastExchange (33) + : +- * BroadcastHashJoin LeftSemi BuildRight (32) : :- * Filter (9) : : +- * ColumnarToRow (8) : : +- Scan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (30) - : +- * Project (29) - : +- * Filter (28) - : +- Window (27) - : +- WindowGroupLimit (26) - : +- * Sort (25) - : +- * HashAggregate (24) - : +- Exchange (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.store_sales (10) - : : +- ReusedExchange (13) - : +- BroadcastExchange (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet spark_catalog.default.store (16) - :- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * HashAggregate (39) - : +- ReusedExchange (38) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- ReusedExchange (43) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Filter (27) + : +- WindowGroupLimit (26) + : +- * Sort (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet spark_catalog.default.store_sales (10) + : : +- ReusedExchange (13) + : +- BroadcastExchange (19) + : +- * Filter (18) + : +- * ColumnarToRow (17) + : +- Scan parquet spark_catalog.default.store (16) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * HashAggregate (40) + : +- ReusedExchange (39) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- ReusedExchange (44) (1) Scan parquet spark_catalog.default.store_sales @@ -65,23 +66,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 61] +(4) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 9] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -89,15 +90,15 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 7] +(8) ColumnarToRow [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(9) Filter [codegen id : 7] +(9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] @@ -114,7 +115,7 @@ Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) -(13) ReusedExchange [Reuses operator id: 61] +(13) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#12] (14) BroadcastHashJoin [codegen id : 4] @@ -181,185 +182,189 @@ Arguments: [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0 Input [3]: [s_state#14, _w0#18, s_state#14] Arguments: [s_state#14], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final -(27) Window +(27) Filter [codegen id : 6] +Input [3]: [s_state#14, _w0#18, s_state#14] +Condition : isnotnull(s_state#14) + +(28) Window Input [3]: [s_state#14, _w0#18, s_state#14] Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST] -(28) Filter [codegen id : 6] +(29) Filter [codegen id : 7] Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] Condition : (ranking#19 <= 5) -(29) Project [codegen id : 6] +(30) Project [codegen id : 7] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#8] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(32) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(35) Project [codegen id : 9] Output [3]: [ss_net_profit#2, s_county#7, s_state#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(35) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 9] Input [3]: [ss_net_profit#2, s_county#7, s_state#8] Keys [2]: [s_state#8, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#20] Results [3]: [s_state#8, s_county#7, sum#21] -(36) Exchange +(37) Exchange Input [3]: [s_state#8, s_county#7, sum#21] Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(37) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 10] Input [3]: [s_state#8, s_county#7, sum#21] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) as decimal(27,2)) AS total_sum#23, s_state#8, s_county#7, 0 AS g_state#24, 0 AS g_county#25, 0 AS lochierarchy#26] -(38) ReusedExchange [Reuses operator id: 36] +(39) ReusedExchange [Reuses operator id: 37] Output [3]: [s_state#8, s_county#7, sum#27] -(39) HashAggregate [codegen id : 18] +(40) HashAggregate [codegen id : 20] Input [3]: [s_state#8, s_county#7, sum#27] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28, s_state#8] -(40) HashAggregate [codegen id : 18] +(41) HashAggregate [codegen id : 20] Input [2]: [total_sum#28, s_state#8] Keys [1]: [s_state#8] Functions [1]: [partial_sum(total_sum#28)] Aggregate Attributes [2]: [sum#29, isEmpty#30] Results [3]: [s_state#8, sum#31, isEmpty#32] -(41) Exchange +(42) Exchange Input [3]: [s_state#8, sum#31, isEmpty#32] Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) HashAggregate [codegen id : 19] +(43) HashAggregate [codegen id : 21] Input [3]: [s_state#8, sum#31, isEmpty#32] Keys [1]: [s_state#8] Functions [1]: [sum(total_sum#28)] Aggregate Attributes [1]: [sum(total_sum#28)#33] Results [6]: [sum(total_sum#28)#33 AS total_sum#34, s_state#8, null AS s_county#35, 0 AS g_state#36, 1 AS g_county#37, 1 AS lochierarchy#38] -(43) ReusedExchange [Reuses operator id: 36] +(44) ReusedExchange [Reuses operator id: 37] Output [3]: [s_state#8, s_county#7, sum#39] -(44) HashAggregate [codegen id : 28] +(45) HashAggregate [codegen id : 31] Input [3]: [s_state#8, s_county#7, sum#39] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28] -(45) HashAggregate [codegen id : 28] +(46) HashAggregate [codegen id : 31] Input [1]: [total_sum#28] Keys: [] Functions [1]: [partial_sum(total_sum#28)] Aggregate Attributes [2]: [sum#40, isEmpty#41] Results [2]: [sum#42, isEmpty#43] -(46) Exchange +(47) Exchange Input [2]: [sum#42, isEmpty#43] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(47) HashAggregate [codegen id : 29] +(48) HashAggregate [codegen id : 32] Input [2]: [sum#42, isEmpty#43] Keys: [] Functions [1]: [sum(total_sum#28)] Aggregate Attributes [1]: [sum(total_sum#28)#44] Results [6]: [sum(total_sum#28)#44 AS total_sum#45, null AS s_state#46, null AS s_county#47, 1 AS g_state#48, 1 AS g_county#49, 2 AS lochierarchy#50] -(48) Union +(49) Union -(49) HashAggregate [codegen id : 30] +(50) HashAggregate [codegen id : 33] Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Functions: [] Aggregate Attributes: [] Results [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] -(50) Exchange +(51) Exchange Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Arguments: hashpartitioning(total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(51) HashAggregate [codegen id : 31] +(52) HashAggregate [codegen id : 34] Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Functions: [] Aggregate Attributes: [] Results [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, CASE WHEN (g_county#25 = 0) THEN s_state#8 END AS _w0#51] -(52) Exchange +(53) Exchange Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51] Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(53) Sort [codegen id : 32] +(54) Sort [codegen id : 35] Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51] Arguments: [lochierarchy#26 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST], false, 0 -(54) Window +(55) Window Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51] Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#26, _w0#51, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#26, _w0#51], [total_sum#23 DESC NULLS LAST] -(55) Project [codegen id : 33] +(56) Project [codegen id : 36] Output [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52] Input [6]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51, rank_within_parent#52] -(56) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52] Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (61) -+- * Project (60) - +- * Filter (59) - +- * ColumnarToRow (58) - +- Scan parquet spark_catalog.default.date_dim (57) +BroadcastExchange (62) ++- * Project (61) + +- * Filter (60) + +- * ColumnarToRow (59) + +- Scan parquet spark_catalog.default.date_dim (58) -(57) Scan parquet spark_catalog.default.date_dim +(58) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#53] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#53] -(59) Filter [codegen id : 1] +(60) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#53] Condition : (((isnotnull(d_month_seq#53) AND (d_month_seq#53 >= 1212)) AND (d_month_seq#53 <= 1223)) AND isnotnull(d_date_sk#5)) -(60) Project [codegen id : 1] +(61) Project [codegen id : 1] Output [1]: [d_date_sk#5] Input [2]: [d_date_sk#5, d_month_seq#53] -(61) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#5] 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-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index 2626c57dd1270..f1d0b073c0210 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (33) + WholeStageCodegen (36) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (32) + WholeStageCodegen (35) Sort [lochierarchy,_w0,total_sum] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (31) + WholeStageCodegen (34) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] InputAdapter Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) + WholeStageCodegen (33) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union - WholeStageCodegen (9) + WholeStageCodegen (10) HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] InputAdapter Exchange [s_state,s_county] #3 - WholeStageCodegen (8) + WholeStageCodegen (9) HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -41,59 +41,62 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (7) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Filter [s_store_sk] - ColumnarToRow + WholeStageCodegen (6) + Filter [s_state] + InputAdapter + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + Exchange [s_state] #7 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Scan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (19) + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (21) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange [s_state] #9 - WholeStageCodegen (18) + WholeStageCodegen (20) HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) + WholeStageCodegen (32) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange #10 - WholeStageCodegen (28) + WholeStageCodegen (31) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 573d9fe01cc58..6d5143bc3c4ea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,60 +1,61 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * Project (55) - +- Window (54) - +- * Sort (53) - +- Exchange (52) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) +TakeOrderedAndProject (57) ++- * Project (56) + +- Window (55) + +- * Sort (54) + +- Exchange (53) + +- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- Union (49) + :- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) : :- * Project (6) : : +- * BroadcastHashJoin Inner BuildRight (5) : : :- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) - : +- BroadcastExchange (32) - : +- * BroadcastHashJoin LeftSemi BuildRight (31) + : +- BroadcastExchange (33) + : +- * BroadcastHashJoin LeftSemi BuildRight (32) : :- * Filter (9) : : +- * ColumnarToRow (8) : : +- Scan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (30) - : +- * Project (29) - : +- * Filter (28) - : +- Window (27) - : +- WindowGroupLimit (26) - : +- * Sort (25) - : +- * HashAggregate (24) - : +- Exchange (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * HashAggregate (39) - : +- ReusedExchange (38) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- ReusedExchange (43) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Filter (27) + : +- WindowGroupLimit (26) + : +- * Sort (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * HashAggregate (40) + : +- ReusedExchange (39) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- ReusedExchange (44) (1) Scan parquet spark_catalog.default.store_sales @@ -65,23 +66,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 9] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(4) ReusedExchange [Reuses operator id: 61] +(4) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 9] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -89,15 +90,15 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 7] +(8) ColumnarToRow [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(9) Filter [codegen id : 7] +(9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] @@ -142,7 +143,7 @@ Join condition: None Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#13] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#13] -(19) ReusedExchange [Reuses operator id: 61] +(19) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date_sk#14] (20) BroadcastHashJoin [codegen id : 4] @@ -181,185 +182,189 @@ Arguments: [s_state#13 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0 Input [3]: [s_state#13, _w0#18, s_state#13] Arguments: [s_state#13], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final -(27) Window +(27) Filter [codegen id : 6] +Input [3]: [s_state#13, _w0#18, s_state#13] +Condition : isnotnull(s_state#13) + +(28) Window Input [3]: [s_state#13, _w0#18, s_state#13] Arguments: [rank(_w0#18) windowspecdefinition(s_state#13, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#13], [_w0#18 DESC NULLS LAST] -(28) Filter [codegen id : 6] +(29) Filter [codegen id : 7] Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19] Condition : (ranking#19 <= 5) -(29) Project [codegen id : 6] +(30) Project [codegen id : 7] Output [1]: [s_state#13] Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#13] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#8] Right keys [1]: [s_state#13] Join type: LeftSemi Join condition: None -(32) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#6, s_county#7, s_state#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(35) Project [codegen id : 9] Output [3]: [ss_net_profit#2, s_county#7, s_state#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] -(35) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 9] Input [3]: [ss_net_profit#2, s_county#7, s_state#8] Keys [2]: [s_state#8, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#20] Results [3]: [s_state#8, s_county#7, sum#21] -(36) Exchange +(37) Exchange Input [3]: [s_state#8, s_county#7, sum#21] Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(37) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 10] Input [3]: [s_state#8, s_county#7, sum#21] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) as decimal(27,2)) AS total_sum#23, s_state#8, s_county#7, 0 AS g_state#24, 0 AS g_county#25, 0 AS lochierarchy#26] -(38) ReusedExchange [Reuses operator id: 36] +(39) ReusedExchange [Reuses operator id: 37] Output [3]: [s_state#8, s_county#7, sum#27] -(39) HashAggregate [codegen id : 18] +(40) HashAggregate [codegen id : 20] Input [3]: [s_state#8, s_county#7, sum#27] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28, s_state#8] -(40) HashAggregate [codegen id : 18] +(41) HashAggregate [codegen id : 20] Input [2]: [total_sum#28, s_state#8] Keys [1]: [s_state#8] Functions [1]: [partial_sum(total_sum#28)] Aggregate Attributes [2]: [sum#29, isEmpty#30] Results [3]: [s_state#8, sum#31, isEmpty#32] -(41) Exchange +(42) Exchange Input [3]: [s_state#8, sum#31, isEmpty#32] Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(42) HashAggregate [codegen id : 19] +(43) HashAggregate [codegen id : 21] Input [3]: [s_state#8, sum#31, isEmpty#32] Keys [1]: [s_state#8] Functions [1]: [sum(total_sum#28)] Aggregate Attributes [1]: [sum(total_sum#28)#33] Results [6]: [sum(total_sum#28)#33 AS total_sum#34, s_state#8, null AS s_county#35, 0 AS g_state#36, 1 AS g_county#37, 1 AS lochierarchy#38] -(43) ReusedExchange [Reuses operator id: 36] +(44) ReusedExchange [Reuses operator id: 37] Output [3]: [s_state#8, s_county#7, sum#39] -(44) HashAggregate [codegen id : 28] +(45) HashAggregate [codegen id : 31] Input [3]: [s_state#8, s_county#7, sum#39] Keys [2]: [s_state#8, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28] -(45) HashAggregate [codegen id : 28] +(46) HashAggregate [codegen id : 31] Input [1]: [total_sum#28] Keys: [] Functions [1]: [partial_sum(total_sum#28)] Aggregate Attributes [2]: [sum#40, isEmpty#41] Results [2]: [sum#42, isEmpty#43] -(46) Exchange +(47) Exchange Input [2]: [sum#42, isEmpty#43] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(47) HashAggregate [codegen id : 29] +(48) HashAggregate [codegen id : 32] Input [2]: [sum#42, isEmpty#43] Keys: [] Functions [1]: [sum(total_sum#28)] Aggregate Attributes [1]: [sum(total_sum#28)#44] Results [6]: [sum(total_sum#28)#44 AS total_sum#45, null AS s_state#46, null AS s_county#47, 1 AS g_state#48, 1 AS g_county#49, 2 AS lochierarchy#50] -(48) Union +(49) Union -(49) HashAggregate [codegen id : 30] +(50) HashAggregate [codegen id : 33] Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Functions: [] Aggregate Attributes: [] Results [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] -(50) Exchange +(51) Exchange Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Arguments: hashpartitioning(total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(51) HashAggregate [codegen id : 31] +(52) HashAggregate [codegen id : 34] Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26] Functions: [] Aggregate Attributes: [] Results [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, CASE WHEN (g_county#25 = 0) THEN s_state#8 END AS _w0#51] -(52) Exchange +(53) Exchange Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51] Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(53) Sort [codegen id : 32] +(54) Sort [codegen id : 35] Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51] Arguments: [lochierarchy#26 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST], false, 0 -(54) Window +(55) Window Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51] Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#26, _w0#51, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#26, _w0#51], [total_sum#23 DESC NULLS LAST] -(55) Project [codegen id : 33] +(56) Project [codegen id : 36] Output [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52] Input [6]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51, rank_within_parent#52] -(56) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52] Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (61) -+- * Project (60) - +- * Filter (59) - +- * ColumnarToRow (58) - +- Scan parquet spark_catalog.default.date_dim (57) +BroadcastExchange (62) ++- * Project (61) + +- * Filter (60) + +- * ColumnarToRow (59) + +- Scan parquet spark_catalog.default.date_dim (58) -(57) Scan parquet spark_catalog.default.date_dim +(58) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#53] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 1] +(59) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#53] -(59) Filter [codegen id : 1] +(60) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_month_seq#53] Condition : (((isnotnull(d_month_seq#53) AND (d_month_seq#53 >= 1212)) AND (d_month_seq#53 <= 1223)) AND isnotnull(d_date_sk#5)) -(60) Project [codegen id : 1] +(61) Project [codegen id : 1] Output [1]: [d_date_sk#5] Input [2]: [d_date_sk#5, d_month_seq#53] -(61) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#5] 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-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 8d0f4d8a9a949..459b5ab9dd70d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (33) + WholeStageCodegen (36) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (32) + WholeStageCodegen (35) Sort [lochierarchy,_w0,total_sum] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (31) + WholeStageCodegen (34) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] InputAdapter Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) + WholeStageCodegen (33) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union - WholeStageCodegen (9) + WholeStageCodegen (10) HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] InputAdapter Exchange [s_state,s_county] #3 - WholeStageCodegen (8) + WholeStageCodegen (9) HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -41,59 +41,62 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk] + Filter [s_store_sk,s_state] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (7) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Filter [s_store_sk] + WholeStageCodegen (6) + Filter [s_state] + InputAdapter + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + Exchange [s_state] #7 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (21) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange [s_state] #9 - WholeStageCodegen (18) + WholeStageCodegen (20) HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) + WholeStageCodegen (32) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange #10 - WholeStageCodegen (28) + WholeStageCodegen (31) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt index eace59b5e4b6e..528be1a965c23 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt @@ -1,34 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) +TakeOrderedAndProject (31) ++- * HashAggregate (30) + +- Exchange (29) + +- * HashAggregate (28) + +- * Project (27) + +- * BroadcastHashJoin Inner BuildRight (26) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) : :- * Filter (3) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.customer (1) - : +- BroadcastExchange (16) - : +- * BroadcastHashJoin LeftSemi BuildRight (15) + : +- BroadcastExchange (17) + : +- * BroadcastHashJoin LeftSemi BuildRight (16) : :- * Filter (6) : : +- * ColumnarToRow (5) : : +- Scan parquet spark_catalog.default.orders (4) - : +- BroadcastExchange (14) - : +- * Project (13) - : +- * Filter (12) - : +- * HashAggregate (11) - : +- Exchange (10) - : +- * HashAggregate (9) - : +- * ColumnarToRow (8) - : +- Scan parquet spark_catalog.default.lineitem (7) - +- BroadcastExchange (24) - +- * BroadcastHashJoin LeftSemi BuildRight (23) - :- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet spark_catalog.default.lineitem (19) - +- ReusedExchange (22) + : +- BroadcastExchange (15) + : +- * Project (14) + : +- * Filter (13) + : +- * HashAggregate (12) + : +- Exchange (11) + : +- * HashAggregate (10) + : +- * Filter (9) + : +- * ColumnarToRow (8) + : +- Scan parquet spark_catalog.default.lineitem (7) + +- BroadcastExchange (25) + +- * BroadcastHashJoin LeftSemi BuildRight (24) + :- * Filter (22) + : +- * ColumnarToRow (21) + : +- Scan parquet spark_catalog.default.lineitem (20) + +- ReusedExchange (23) (1) Scan parquet spark_catalog.default.customer @@ -63,117 +64,122 @@ Condition : (isnotnull(o_custkey#4) AND isnotnull(o_orderkey#3)) Output [2]: [l_orderkey#7, l_quantity#8] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] +PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] Input [2]: [l_orderkey#7, l_quantity#8] -(9) HashAggregate [codegen id : 1] +(9) Filter [codegen id : 1] +Input [2]: [l_orderkey#7, l_quantity#8] +Condition : isnotnull(l_orderkey#7) + +(10) HashAggregate [codegen id : 1] Input [2]: [l_orderkey#7, l_quantity#8] Keys [1]: [l_orderkey#7] Functions [1]: [partial_sum(l_quantity#8)] Aggregate Attributes [2]: [sum#9, isEmpty#10] Results [3]: [l_orderkey#7, sum#11, isEmpty#12] -(10) Exchange +(11) Exchange Input [3]: [l_orderkey#7, sum#11, isEmpty#12] Arguments: hashpartitioning(l_orderkey#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(11) HashAggregate [codegen id : 2] +(12) HashAggregate [codegen id : 2] Input [3]: [l_orderkey#7, sum#11, isEmpty#12] Keys [1]: [l_orderkey#7] Functions [1]: [sum(l_quantity#8)] Aggregate Attributes [1]: [sum(l_quantity#8)#13] Results [2]: [l_orderkey#7, sum(l_quantity#8)#13 AS sum(l_quantity#14)#15] -(12) Filter [codegen id : 2] +(13) Filter [codegen id : 2] Input [2]: [l_orderkey#7, sum(l_quantity#14)#15] Condition : (isnotnull(sum(l_quantity#14)#15) AND (sum(l_quantity#14)#15 > 300)) -(13) Project [codegen id : 2] +(14) Project [codegen id : 2] Output [1]: [l_orderkey#7] Input [2]: [l_orderkey#7, sum(l_quantity#14)#15] -(14) BroadcastExchange +(15) BroadcastExchange Input [1]: [l_orderkey#7] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 3] +(16) BroadcastHashJoin [codegen id : 3] Left keys [1]: [o_orderkey#3] Right keys [1]: [l_orderkey#7] Join type: LeftSemi Join condition: None -(16) BroadcastExchange +(17) BroadcastExchange Input [4]: [o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false]),false), [plan_id=3] -(17) BroadcastHashJoin [codegen id : 7] +(18) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_custkey#1] Right keys [1]: [o_custkey#4] Join type: Inner Join condition: None -(18) Project [codegen id : 7] +(19) Project [codegen id : 7] Output [5]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6] Input [6]: [c_custkey#1, c_name#2, o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] -(19) Scan parquet spark_catalog.default.lineitem +(20) Scan parquet spark_catalog.default.lineitem Output [2]: [l_orderkey#16, l_quantity#14] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [2]: [l_orderkey#16, l_quantity#14] -(21) Filter [codegen id : 6] +(22) Filter [codegen id : 6] Input [2]: [l_orderkey#16, l_quantity#14] Condition : isnotnull(l_orderkey#16) -(22) ReusedExchange [Reuses operator id: 14] +(23) ReusedExchange [Reuses operator id: 15] Output [1]: [l_orderkey#7] -(23) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [l_orderkey#16] Right keys [1]: [l_orderkey#7] Join type: LeftSemi Join condition: None -(24) BroadcastExchange +(25) BroadcastExchange Input [2]: [l_orderkey#16, l_quantity#14] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 7] +(26) BroadcastHashJoin [codegen id : 7] Left keys [1]: [o_orderkey#3] Right keys [1]: [l_orderkey#16] Join type: Inner Join condition: None -(26) Project [codegen id : 7] +(27) Project [codegen id : 7] Output [6]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6, l_quantity#14] Input [7]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6, l_orderkey#16, l_quantity#14] -(27) HashAggregate [codegen id : 7] +(28) HashAggregate [codegen id : 7] Input [6]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6, l_quantity#14] Keys [5]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5] Functions [1]: [partial_sum(l_quantity#14)] Aggregate Attributes [2]: [sum#17, isEmpty#18] Results [7]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum#19, isEmpty#20] -(28) Exchange +(29) Exchange Input [7]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum#19, isEmpty#20] Arguments: hashpartitioning(c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(29) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 8] Input [7]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum#19, isEmpty#20] Keys [5]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5] Functions [1]: [sum(l_quantity#14)] Aggregate Attributes [1]: [sum(l_quantity#14)#21] Results [6]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum(l_quantity#14)#21 AS sum(l_quantity)#22] -(30) TakeOrderedAndProject +(31) TakeOrderedAndProject Input [6]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum(l_quantity)#22] Arguments: 100, [o_totalprice#5 DESC NULLS LAST, o_orderdate#6 ASC NULLS FIRST], [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum(l_quantity)#22] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt index c809a4bbec73d..dcc3345a18821 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt @@ -31,9 +31,10 @@ TakeOrderedAndProject [o_totalprice,o_orderdate,c_name,c_custkey,o_orderkey,sum( Exchange [l_orderkey] #4 WholeStageCodegen (1) HashAggregate [l_orderkey,l_quantity] [sum,isEmpty,sum,isEmpty] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.lineitem [l_orderkey,l_quantity] + Filter [l_orderkey] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.lineitem [l_orderkey,l_quantity] InputAdapter BroadcastExchange #5 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt index 1f8efaaf1a3a0..558e9adab6196 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt @@ -36,7 +36,7 @@ Input [6]: [l_partkey#1, l_quantity#2, l_extendedprice#3, l_discount#4, l_shipin Output [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] Batched: true Location [not included in comparison]/{warehouse_dir}/part] -PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#11),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#12),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#13),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] +PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#11),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#6),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#12),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] (7) Filter [codegen id : 1] Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] -Condition : (((isnotnull(p_size#9) AND (p_size#9 >= 1)) AND isnotnull(p_partkey#7)) AND (((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#9 <= 5)) OR (((p_brand#8 = Brand#12) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#9 <= 10))) OR (((p_brand#8 = Brand#13) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#9 <= 15)))) +Condition : (((isnotnull(p_size#9) AND (p_size#9 >= 1)) AND isnotnull(p_partkey#7)) AND (((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#9 <= 5)) OR (((p_brand#8 = Brand#6) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#9 <= 10))) OR (((p_brand#8 = Brand#12) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#9 <= 15)))) (8) BroadcastExchange Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] @@ -54,7 +54,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [pl Left keys [1]: [l_partkey#1] Right keys [1]: [p_partkey#7] Join type: Inner -Join condition: (((((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#2 >= 1)) AND (l_quantity#2 <= 11)) AND (p_size#9 <= 5)) OR (((((p_brand#8 = Brand#12) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#2 >= 10)) AND (l_quantity#2 <= 20)) AND (p_size#9 <= 10))) OR (((((p_brand#8 = Brand#13) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#2 >= 20)) AND (l_quantity#2 <= 30)) AND (p_size#9 <= 15))) +Join condition: (((((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#2 >= 1)) AND (l_quantity#2 <= 11)) AND (p_size#9 <= 5)) OR (((((p_brand#8 = Brand#6) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#2 >= 10)) AND (l_quantity#2 <= 20)) AND (p_size#9 <= 10))) OR (((((p_brand#8 = Brand#12) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#2 >= 20)) AND (l_quantity#2 <= 30)) AND (p_size#9 <= 15))) (10) Project [codegen id : 2] Output [2]: [l_extendedprice#3, l_discount#4] @@ -64,17 +64,17 @@ Input [8]: [l_partkey#1, l_quantity#2, l_extendedprice#3, l_discount#4, p_partke Input [2]: [l_extendedprice#3, l_discount#4] Keys: [] Functions [1]: [partial_sum((l_extendedprice#3 * (1 - l_discount#4)))] -Aggregate Attributes [2]: [sum#14, isEmpty#15] -Results [2]: [sum#16, isEmpty#17] +Aggregate Attributes [2]: [sum#13, isEmpty#14] +Results [2]: [sum#15, isEmpty#16] (12) Exchange -Input [2]: [sum#16, isEmpty#17] +Input [2]: [sum#15, isEmpty#16] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] (13) HashAggregate [codegen id : 3] -Input [2]: [sum#16, isEmpty#17] +Input [2]: [sum#15, isEmpty#16] Keys: [] Functions [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))] -Aggregate Attributes [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#18] -Results [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#18 AS revenue#19] +Aggregate Attributes [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#17] +Results [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#17 AS revenue#18] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt index 9158702d26579..5f6ccb95b376a 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt @@ -42,7 +42,7 @@ Output [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] Batched: true Location [not included in comparison]/{warehouse_dir}/supplier] -PushedFilters: [IsNotNull(s_nationkey)] +PushedFilters: [IsNotNull(s_nationkey), IsNotNull(s_suppkey)] ReadSchema: struct (2) ColumnarToRow [codegen id : 7] @@ -50,7 +50,7 @@ Input [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] (3) Filter [codegen id : 7] Input [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] -Condition : isnotnull(s_nationkey#4) +Condition : (isnotnull(s_nationkey#4) AND isnotnull(s_suppkey#1)) (4) Scan parquet spark_catalog.default.partsupp Output [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] @@ -70,7 +70,7 @@ Condition : ((isnotnull(ps_availqty#7) AND isnotnull(ps_partkey#5)) AND isnotnul Output [2]: [p_partkey#8, p_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/part] -PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] +PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest), IsNotNull(p_partkey)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] @@ -78,7 +78,7 @@ Input [2]: [p_partkey#8, p_name#9] (9) Filter [codegen id : 1] Input [2]: [p_partkey#8, p_name#9] -Condition : (isnotnull(p_name#9) AND StartsWith(p_name#9, forest)) +Condition : ((isnotnull(p_name#9) AND StartsWith(p_name#9, forest)) AND isnotnull(p_partkey#8)) (10) Project [codegen id : 1] Output [1]: [p_partkey#8] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt index 80c8e514fb3eb..6cc10adcfb20c 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (8) BroadcastHashJoin [s_nationkey,n_nationkey] Project [s_name,s_address,s_nationkey] BroadcastHashJoin [s_suppkey,ps_suppkey] - Filter [s_nationkey] + Filter [s_nationkey,s_suppkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.supplier [s_suppkey,s_name,s_address,s_nationkey] @@ -25,7 +25,7 @@ WholeStageCodegen (8) BroadcastExchange #3 WholeStageCodegen (1) Project [p_partkey] - Filter [p_name] + Filter [p_name,p_partkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.part [p_partkey,p_name] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt index c57939f093729..d9203026e7da7 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt @@ -1,42 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) +TakeOrderedAndProject (39) ++- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) : : :- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.supplier (1) - : : +- BroadcastExchange (18) - : : +- * BroadcastHashJoin LeftAnti BuildRight (17) - : : :- * BroadcastHashJoin LeftSemi BuildRight (11) + : : +- BroadcastExchange (19) + : : +- * BroadcastHashJoin LeftAnti BuildRight (18) + : : :- * BroadcastHashJoin LeftSemi BuildRight (12) : : : :- * Project (7) : : : : +- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.lineitem (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- Scan parquet spark_catalog.default.lineitem (8) - : : +- BroadcastExchange (16) - : : +- * Project (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet spark_catalog.default.lineitem (12) - : +- BroadcastExchange (25) - : +- * Project (24) - : +- * Filter (23) - : +- * ColumnarToRow (22) - : +- Scan parquet spark_catalog.default.orders (21) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * ColumnarToRow (29) - +- Scan parquet spark_catalog.default.nation (28) + : : : +- BroadcastExchange (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet spark_catalog.default.lineitem (8) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet spark_catalog.default.lineitem (13) + : +- BroadcastExchange (26) + : +- * Project (25) + : +- * Filter (24) + : +- * ColumnarToRow (23) + : +- Scan parquet spark_catalog.default.orders (22) + +- BroadcastExchange (33) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet spark_catalog.default.nation (29) (1) Scan parquet spark_catalog.default.supplier @@ -75,146 +76,151 @@ Input [4]: [l_orderkey#4, l_suppkey#5, l_commitdate#6, l_receiptdate#7] Output [2]: [l_orderkey#8, l_suppkey#9] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] +PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct (9) ColumnarToRow [codegen id : 1] Input [2]: [l_orderkey#8, l_suppkey#9] -(10) BroadcastExchange +(10) Filter [codegen id : 1] Input [2]: [l_orderkey#8, l_suppkey#9] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=1] +Condition : (isnotnull(l_orderkey#8) AND isnotnull(l_suppkey#9)) -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastExchange +Input [2]: [l_orderkey#8, l_suppkey#9] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=1] + +(12) BroadcastHashJoin [codegen id : 3] Left keys [1]: [l_orderkey#4] Right keys [1]: [l_orderkey#8] Join type: LeftSemi Join condition: NOT (l_suppkey#9 = l_suppkey#5) -(12) Scan parquet spark_catalog.default.lineitem +(13) Scan parquet spark_catalog.default.lineitem Output [4]: [l_orderkey#10, l_suppkey#11, l_commitdate#12, l_receiptdate#13] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] -PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate)] +PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 2] +(14) ColumnarToRow [codegen id : 2] Input [4]: [l_orderkey#10, l_suppkey#11, l_commitdate#12, l_receiptdate#13] -(14) Filter [codegen id : 2] +(15) Filter [codegen id : 2] Input [4]: [l_orderkey#10, l_suppkey#11, l_commitdate#12, l_receiptdate#13] -Condition : ((isnotnull(l_receiptdate#13) AND isnotnull(l_commitdate#12)) AND (l_receiptdate#13 > l_commitdate#12)) +Condition : ((((isnotnull(l_receiptdate#13) AND isnotnull(l_commitdate#12)) AND (l_receiptdate#13 > l_commitdate#12)) AND isnotnull(l_orderkey#10)) AND isnotnull(l_suppkey#11)) -(15) Project [codegen id : 2] +(16) Project [codegen id : 2] Output [2]: [l_orderkey#10, l_suppkey#11] Input [4]: [l_orderkey#10, l_suppkey#11, l_commitdate#12, l_receiptdate#13] -(16) BroadcastExchange +(17) BroadcastExchange Input [2]: [l_orderkey#10, l_suppkey#11] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 3] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [l_orderkey#4] Right keys [1]: [l_orderkey#10] Join type: LeftAnti Join condition: NOT (l_suppkey#11 = l_suppkey#5) -(18) BroadcastExchange +(19) BroadcastExchange Input [2]: [l_orderkey#4, l_suppkey#5] Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 6] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_suppkey#1] Right keys [1]: [l_suppkey#5] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(21) Project [codegen id : 6] Output [3]: [s_name#2, s_nationkey#3, l_orderkey#4] Input [5]: [s_suppkey#1, s_name#2, s_nationkey#3, l_orderkey#4, l_suppkey#5] -(21) Scan parquet spark_catalog.default.orders +(22) Scan parquet spark_catalog.default.orders Output [2]: [o_orderkey#14, o_orderstatus#15] Batched: true Location [not included in comparison]/{warehouse_dir}/orders] PushedFilters: [IsNotNull(o_orderstatus), EqualTo(o_orderstatus,F), IsNotNull(o_orderkey)] ReadSchema: struct -(22) ColumnarToRow [codegen id : 4] +(23) ColumnarToRow [codegen id : 4] Input [2]: [o_orderkey#14, o_orderstatus#15] -(23) Filter [codegen id : 4] +(24) Filter [codegen id : 4] Input [2]: [o_orderkey#14, o_orderstatus#15] Condition : ((isnotnull(o_orderstatus#15) AND (o_orderstatus#15 = F)) AND isnotnull(o_orderkey#14)) -(24) Project [codegen id : 4] +(25) Project [codegen id : 4] Output [1]: [o_orderkey#14] Input [2]: [o_orderkey#14, o_orderstatus#15] -(25) BroadcastExchange +(26) BroadcastExchange Input [1]: [o_orderkey#14] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=4] -(26) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [l_orderkey#4] Right keys [1]: [o_orderkey#14] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(28) Project [codegen id : 6] Output [2]: [s_name#2, s_nationkey#3] Input [4]: [s_name#2, s_nationkey#3, l_orderkey#4, o_orderkey#14] -(28) Scan parquet spark_catalog.default.nation +(29) Scan parquet spark_catalog.default.nation Output [2]: [n_nationkey#16, n_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/nation] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,SAUDI ARABIA), IsNotNull(n_nationkey)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 5] +(30) ColumnarToRow [codegen id : 5] Input [2]: [n_nationkey#16, n_name#17] -(30) Filter [codegen id : 5] +(31) Filter [codegen id : 5] Input [2]: [n_nationkey#16, n_name#17] Condition : ((isnotnull(n_name#17) AND (n_name#17 = SAUDI ARABIA)) AND isnotnull(n_nationkey#16)) -(31) Project [codegen id : 5] +(32) Project [codegen id : 5] Output [1]: [n_nationkey#16] Input [2]: [n_nationkey#16, n_name#17] -(32) BroadcastExchange +(33) BroadcastExchange Input [1]: [n_nationkey#16] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 6] +(34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_nationkey#3] Right keys [1]: [n_nationkey#16] Join type: Inner Join condition: None -(34) Project [codegen id : 6] +(35) Project [codegen id : 6] Output [1]: [s_name#2] Input [3]: [s_name#2, s_nationkey#3, n_nationkey#16] -(35) HashAggregate [codegen id : 6] +(36) HashAggregate [codegen id : 6] Input [1]: [s_name#2] Keys [1]: [s_name#2] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] Results [2]: [s_name#2, count#19] -(36) Exchange +(37) Exchange Input [2]: [s_name#2, count#19] Arguments: hashpartitioning(s_name#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 7] Input [2]: [s_name#2, count#19] Keys [1]: [s_name#2] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [s_name#2, count(1)#20 AS numwait#21] -(38) TakeOrderedAndProject +(39) TakeOrderedAndProject Input [2]: [s_name#2, numwait#21] Arguments: 100, [numwait#21 DESC NULLS LAST, s_name#2 ASC NULLS FIRST], [s_name#2, numwait#21] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt index cf358cf245b19..4687bf94b96cf 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt @@ -28,14 +28,15 @@ TakeOrderedAndProject [numwait,s_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.lineitem [l_orderkey,l_suppkey] + Filter [l_orderkey,l_suppkey] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.lineitem [l_orderkey,l_suppkey] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [l_orderkey,l_suppkey] - Filter [l_receiptdate,l_commitdate] + Filter [l_receiptdate,l_commitdate,l_orderkey,l_suppkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.lineitem [l_orderkey,l_suppkey,l_commitdate,l_receiptdate] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q22/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q22/explain.txt index 14405dab7bf78..f37ffb9fec846 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q22/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q22/explain.txt @@ -1,17 +1,18 @@ == Physical Plan == -* Sort (13) -+- Exchange (12) - +- * HashAggregate (11) - +- Exchange (10) - +- * HashAggregate (9) - +- * Project (8) - +- * BroadcastHashJoin LeftAnti BuildRight (7) +* Sort (14) ++- Exchange (13) + +- * HashAggregate (12) + +- Exchange (11) + +- * HashAggregate (10) + +- * Project (9) + +- * BroadcastHashJoin LeftAnti BuildRight (8) :- * Filter (3) : +- * ColumnarToRow (2) : +- Scan parquet spark_catalog.default.customer (1) - +- BroadcastExchange (6) - +- * ColumnarToRow (5) - +- Scan parquet spark_catalog.default.orders (4) + +- BroadcastExchange (7) + +- * Filter (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.orders (4) (1) Scan parquet spark_catalog.default.customer @@ -32,93 +33,98 @@ Condition : ((isnotnull(c_acctbal#3) AND substring(c_phone#2, 1, 2) IN (13,31,23 Output [1]: [o_custkey#6] Batched: true Location [not included in comparison]/{warehouse_dir}/orders] +PushedFilters: [IsNotNull(o_custkey)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input [1]: [o_custkey#6] -(6) BroadcastExchange +(6) Filter [codegen id : 1] Input [1]: [o_custkey#6] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=1] +Condition : isnotnull(o_custkey#6) -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastExchange +Input [1]: [o_custkey#6] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_custkey#1] Right keys [1]: [o_custkey#6] Join type: LeftAnti Join condition: None -(8) Project [codegen id : 2] +(9) Project [codegen id : 2] Output [2]: [substring(c_phone#2, 1, 2) AS cntrycode#7, c_acctbal#3] Input [3]: [c_custkey#1, c_phone#2, c_acctbal#3] -(9) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 2] Input [2]: [cntrycode#7, c_acctbal#3] Keys [1]: [cntrycode#7] Functions [2]: [partial_count(1), partial_sum(c_acctbal#3)] Aggregate Attributes [3]: [count#8, sum#9, isEmpty#10] Results [4]: [cntrycode#7, count#11, sum#12, isEmpty#13] -(10) Exchange +(11) Exchange Input [4]: [cntrycode#7, count#11, sum#12, isEmpty#13] Arguments: hashpartitioning(cntrycode#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) HashAggregate [codegen id : 3] +(12) HashAggregate [codegen id : 3] Input [4]: [cntrycode#7, count#11, sum#12, isEmpty#13] Keys [1]: [cntrycode#7] Functions [2]: [count(1), sum(c_acctbal#3)] Aggregate Attributes [2]: [count(1)#14, sum(c_acctbal#3)#15] Results [3]: [cntrycode#7, count(1)#14 AS numcust#16, sum(c_acctbal#3)#15 AS totacctbal#17] -(12) Exchange +(13) Exchange Input [3]: [cntrycode#7, numcust#16, totacctbal#17] Arguments: rangepartitioning(cntrycode#7 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(13) Sort [codegen id : 4] +(14) Sort [codegen id : 4] Input [3]: [cntrycode#7, numcust#16, totacctbal#17] Arguments: [cntrycode#7 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#4, [id=#5] -* HashAggregate (20) -+- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * Filter (16) - +- * ColumnarToRow (15) - +- Scan parquet spark_catalog.default.customer (14) +* HashAggregate (21) ++- Exchange (20) + +- * HashAggregate (19) + +- * Project (18) + +- * Filter (17) + +- * ColumnarToRow (16) + +- Scan parquet spark_catalog.default.customer (15) -(14) Scan parquet spark_catalog.default.customer +(15) Scan parquet spark_catalog.default.customer Output [2]: [c_phone#18, c_acctbal#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_acctbal), GreaterThan(c_acctbal,0)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 1] +(16) ColumnarToRow [codegen id : 1] Input [2]: [c_phone#18, c_acctbal#19] -(16) Filter [codegen id : 1] +(17) Filter [codegen id : 1] Input [2]: [c_phone#18, c_acctbal#19] Condition : ((isnotnull(c_acctbal#19) AND (c_acctbal#19 > 0)) AND substring(c_phone#18, 1, 2) IN (13,31,23,29,30,18,17)) -(17) Project [codegen id : 1] +(18) Project [codegen id : 1] Output [1]: [c_acctbal#19] Input [2]: [c_phone#18, c_acctbal#19] -(18) HashAggregate [codegen id : 1] +(19) HashAggregate [codegen id : 1] Input [1]: [c_acctbal#19] Keys: [] Functions [1]: [partial_avg(UnscaledValue(c_acctbal#19))] Aggregate Attributes [2]: [sum#20, count#21] Results [2]: [sum#22, count#23] -(19) Exchange +(20) Exchange Input [2]: [sum#22, count#23] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(20) HashAggregate [codegen id : 2] +(21) HashAggregate [codegen id : 2] Input [2]: [sum#22, count#23] Keys: [] Functions [1]: [avg(UnscaledValue(c_acctbal#19))] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q22/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q22/simplified.txt index 51e531a1e8a46..c9f5ef44fbca9 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q22/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q22/simplified.txt @@ -29,6 +29,7 @@ WholeStageCodegen (4) InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.orders [o_custkey] + Filter [o_custkey] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.orders [o_custkey] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q4/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q4/explain.txt index 2c4aa5f649ba3..90ab3ffeaf86b 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q4/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q4/explain.txt @@ -21,7 +21,7 @@ Output [3]: [o_orderkey#1, o_orderdate#2, o_orderpriority#3] Batched: true Location [not included in comparison]/{warehouse_dir}/orders] -PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01)] +PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1993-07-01), LessThan(o_orderdate,1993-10-01), IsNotNull(o_orderkey)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] @@ -29,7 +29,7 @@ Input [3]: [o_orderkey#1, o_orderdate#2, o_orderpriority#3] (3) Filter [codegen id : 2] Input [3]: [o_orderkey#1, o_orderdate#2, o_orderpriority#3] -Condition : ((isnotnull(o_orderdate#2) AND (o_orderdate#2 >= 1993-07-01)) AND (o_orderdate#2 < 1993-10-01)) +Condition : (((isnotnull(o_orderdate#2) AND (o_orderdate#2 >= 1993-07-01)) AND (o_orderdate#2 < 1993-10-01)) AND isnotnull(o_orderkey#1)) (4) Project [codegen id : 2] Output [2]: [o_orderkey#1, o_orderpriority#3] @@ -39,7 +39,7 @@ Input [3]: [o_orderkey#1, o_orderdate#2, o_orderpriority#3] Output [3]: [l_orderkey#4, l_commitdate#5, l_receiptdate#6] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] -PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate)] +PushedFilters: [IsNotNull(l_commitdate), IsNotNull(l_receiptdate), IsNotNull(l_orderkey)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -47,7 +47,7 @@ Input [3]: [l_orderkey#4, l_commitdate#5, l_receiptdate#6] (7) Filter [codegen id : 1] Input [3]: [l_orderkey#4, l_commitdate#5, l_receiptdate#6] -Condition : ((isnotnull(l_commitdate#5) AND isnotnull(l_receiptdate#6)) AND (l_commitdate#5 < l_receiptdate#6)) +Condition : (((isnotnull(l_commitdate#5) AND isnotnull(l_receiptdate#6)) AND (l_commitdate#5 < l_receiptdate#6)) AND isnotnull(l_orderkey#4)) (8) Project [codegen id : 1] Output [1]: [l_orderkey#4] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q4/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q4/simplified.txt index ff099b6a0dc8b..bb43977bffee3 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q4/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q4/simplified.txt @@ -11,7 +11,7 @@ WholeStageCodegen (4) Project [o_orderpriority] BroadcastHashJoin [o_orderkey,l_orderkey] Project [o_orderkey,o_orderpriority] - Filter [o_orderdate] + Filter [o_orderdate,o_orderkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.orders [o_orderkey,o_orderdate,o_orderpriority] @@ -19,7 +19,7 @@ WholeStageCodegen (4) BroadcastExchange #3 WholeStageCodegen (1) Project [l_orderkey] - Filter [l_commitdate,l_receiptdate] + Filter [l_commitdate,l_receiptdate,l_orderkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.lineitem [l_orderkey,l_commitdate,l_receiptdate] From f524af75892987d72087d3f7ad052501c83490c2 Mon Sep 17 00:00:00 2001 From: Kapil Kumar Singh Date: Mon, 6 Mar 2023 10:15:33 +0530 Subject: [PATCH 2/6] Fix test --- .../tpch-plan-stability/q19/explain.txt | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt index 558e9adab6196..1f8efaaf1a3a0 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt @@ -36,7 +36,7 @@ Input [6]: [l_partkey#1, l_quantity#2, l_extendedprice#3, l_discount#4, l_shipin Output [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] Batched: true Location [not included in comparison]/{warehouse_dir}/part] -PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#11),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#6),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#12),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] +PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#11),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#12),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#13),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] (7) Filter [codegen id : 1] Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] -Condition : (((isnotnull(p_size#9) AND (p_size#9 >= 1)) AND isnotnull(p_partkey#7)) AND (((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#9 <= 5)) OR (((p_brand#8 = Brand#6) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#9 <= 10))) OR (((p_brand#8 = Brand#12) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#9 <= 15)))) +Condition : (((isnotnull(p_size#9) AND (p_size#9 >= 1)) AND isnotnull(p_partkey#7)) AND (((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#9 <= 5)) OR (((p_brand#8 = Brand#12) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#9 <= 10))) OR (((p_brand#8 = Brand#13) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#9 <= 15)))) (8) BroadcastExchange Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] @@ -54,7 +54,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [pl Left keys [1]: [l_partkey#1] Right keys [1]: [p_partkey#7] Join type: Inner -Join condition: (((((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#2 >= 1)) AND (l_quantity#2 <= 11)) AND (p_size#9 <= 5)) OR (((((p_brand#8 = Brand#6) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#2 >= 10)) AND (l_quantity#2 <= 20)) AND (p_size#9 <= 10))) OR (((((p_brand#8 = Brand#12) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#2 >= 20)) AND (l_quantity#2 <= 30)) AND (p_size#9 <= 15))) +Join condition: (((((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#2 >= 1)) AND (l_quantity#2 <= 11)) AND (p_size#9 <= 5)) OR (((((p_brand#8 = Brand#12) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#2 >= 10)) AND (l_quantity#2 <= 20)) AND (p_size#9 <= 10))) OR (((((p_brand#8 = Brand#13) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#2 >= 20)) AND (l_quantity#2 <= 30)) AND (p_size#9 <= 15))) (10) Project [codegen id : 2] Output [2]: [l_extendedprice#3, l_discount#4] @@ -64,17 +64,17 @@ Input [8]: [l_partkey#1, l_quantity#2, l_extendedprice#3, l_discount#4, p_partke Input [2]: [l_extendedprice#3, l_discount#4] Keys: [] Functions [1]: [partial_sum((l_extendedprice#3 * (1 - l_discount#4)))] -Aggregate Attributes [2]: [sum#13, isEmpty#14] -Results [2]: [sum#15, isEmpty#16] +Aggregate Attributes [2]: [sum#14, isEmpty#15] +Results [2]: [sum#16, isEmpty#17] (12) Exchange -Input [2]: [sum#15, isEmpty#16] +Input [2]: [sum#16, isEmpty#17] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] (13) HashAggregate [codegen id : 3] -Input [2]: [sum#15, isEmpty#16] +Input [2]: [sum#16, isEmpty#17] Keys: [] Functions [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))] -Aggregate Attributes [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#17] -Results [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#17 AS revenue#18] +Aggregate Attributes [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#18] +Results [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#18 AS revenue#19] From 22e7886ff1059b98d1525380b2cb22718fd5dd09 Mon Sep 17 00:00:00 2001 From: Kapil Kumar Singh Date: Fri, 17 Mar 2023 17:13:13 +0530 Subject: [PATCH 3/6] Move rule before --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index cb63592608869..f4d4103b8f5f9 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 @@ -176,6 +176,8 @@ abstract class Optimizer(catalogManager: CatalogManager) // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). Batch("Subquery", FixedPoint(1), OptimizeSubqueries) :: + Batch("RewriteSubquery", Once, + RewritePredicateSubquery) :: Batch("Replace Operators", fixedPoint, RewriteExceptAll, RewriteIntersectAll, @@ -224,16 +226,6 @@ abstract class Optimizer(catalogManager: CatalogManager) // The following batch should be executed after batch "Join Reorder" and "LocalRelation". Batch("Check Cartesian Products", Once, CheckCartesianProducts) :+ - Batch("RewriteSubquery", Once, - RewritePredicateSubquery, - PushPredicateThroughJoin, - LimitPushDown, - ColumnPruning, - CollapseProject, - RemoveRedundantAliases, - RemoveNoopOperators, - InferFiltersFromConstraints, - PushDownPredicates) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+ Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression) From 58fcfbcbf3d44498fb382bd384343bd08bed716e Mon Sep 17 00:00:00 2001 From: Kapil Kumar Singh Date: Mon, 20 Mar 2023 12:41:19 +0530 Subject: [PATCH 4/6] Update plans --- .../approved-plans-v1_4/q10.sf100/explain.txt | 439 ++++++----- .../q10.sf100/simplified.txt | 201 +++-- .../approved-plans-v1_4/q10/explain.txt | 300 ++++---- .../approved-plans-v1_4/q10/simplified.txt | 74 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 282 +++---- .../q11.sf100/simplified.txt | 112 +-- .../approved-plans-v1_4/q11/explain.txt | 254 ++++--- .../approved-plans-v1_4/q11/simplified.txt | 96 +-- .../q14a.sf100/explain.txt | 373 +++++---- .../q14a.sf100/simplified.txt | 36 +- .../approved-plans-v1_4/q14a/explain.txt | 625 ++++++++------- .../approved-plans-v1_4/q14a/simplified.txt | 122 ++- .../q14b.sf100/explain.txt | 327 ++++---- .../q14b.sf100/simplified.txt | 21 +- .../approved-plans-v1_4/q14b/explain.txt | 579 +++++++------- .../approved-plans-v1_4/q14b/simplified.txt | 107 ++- .../approved-plans-v1_4/q16.sf100/explain.txt | 4 +- .../q16.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q16/explain.txt | 4 +- .../approved-plans-v1_4/q16/simplified.txt | 2 +- .../q23b.sf100/explain.txt | 496 +++++------- .../q23b.sf100/simplified.txt | 102 +-- .../approved-plans-v1_4/q23b/explain.txt | 348 ++++----- .../approved-plans-v1_4/q23b/simplified.txt | 48 +- .../approved-plans-v1_4/q31.sf100/explain.txt | 291 +++---- .../q31.sf100/simplified.txt | 331 ++++---- .../approved-plans-v1_4/q33.sf100/explain.txt | 4 +- .../q33.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q33/explain.txt | 4 +- .../approved-plans-v1_4/q33/simplified.txt | 2 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 416 +++++----- .../q35.sf100/simplified.txt | 198 ++--- .../approved-plans-v1_4/q35/explain.txt | 296 ++++---- .../approved-plans-v1_4/q35/simplified.txt | 72 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 644 ++++++++-------- .../q4.sf100/simplified.txt | 224 +++--- .../approved-plans-v1_4/q4/explain.txt | 580 +++++++------- .../approved-plans-v1_4/q4/simplified.txt | 180 ++--- .../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/q56.sf100/explain.txt | 4 +- .../q56.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q56/explain.txt | 4 +- .../approved-plans-v1_4/q56/simplified.txt | 2 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 4 +- .../q58.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 4 +- .../approved-plans-v1_4/q58/simplified.txt | 2 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 4 +- .../q60.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q60/explain.txt | 4 +- .../approved-plans-v1_4/q60/simplified.txt | 2 +- .../approved-plans-v1_4/q62.sf100/explain.txt | 79 +- .../q62.sf100/simplified.txt | 73 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 4 +- .../q69.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q69/explain.txt | 4 +- .../approved-plans-v1_4/q69/simplified.txt | 2 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 4 +- .../q70.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 4 +- .../approved-plans-v1_4/q70/simplified.txt | 2 +- .../approved-plans-v1_4/q74.sf100/explain.txt | 282 +++---- .../q74.sf100/simplified.txt | 112 +-- .../approved-plans-v1_4/q74/explain.txt | 254 ++++--- .../approved-plans-v1_4/q74/simplified.txt | 96 +-- .../approved-plans-v1_4/q76.sf100/explain.txt | 167 ++-- .../q76.sf100/simplified.txt | 105 +-- .../approved-plans-v1_4/q8.sf100/explain.txt | 3 +- .../approved-plans-v1_4/q8/explain.txt | 3 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 4 +- .../q83.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q83/explain.txt | 4 +- .../approved-plans-v1_4/q83/simplified.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 55 +- .../q93.sf100/simplified.txt | 60 +- .../approved-plans-v1_4/q93/explain.txt | 98 ++- .../approved-plans-v1_4/q93/simplified.txt | 7 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 4 +- .../q94.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q94/explain.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 2 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 4 +- .../q95.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q95/explain.txt | 4 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../approved-plans-v1_4/q99.sf100/explain.txt | 79 +- .../q99.sf100/simplified.txt | 73 +- .../q10a.sf100/explain.txt | 4 +- .../q10a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q10a/explain.txt | 4 +- .../approved-plans-v2_7/q10a/simplified.txt | 2 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 282 +++---- .../q11.sf100/simplified.txt | 112 +-- .../approved-plans-v2_7/q11/explain.txt | 254 ++++--- .../approved-plans-v2_7/q11/simplified.txt | 96 +-- .../approved-plans-v2_7/q14.sf100/explain.txt | 327 ++++---- .../q14.sf100/simplified.txt | 21 +- .../approved-plans-v2_7/q14/explain.txt | 579 +++++++------- .../approved-plans-v2_7/q14/simplified.txt | 107 ++- .../q14a.sf100/explain.txt | 465 ++++++------ .../q14a.sf100/simplified.txt | 70 +- .../approved-plans-v2_7/q14a/explain.txt | 717 +++++++++--------- .../approved-plans-v2_7/q14a/simplified.txt | 156 ++-- .../q18a.sf100/explain.txt | 590 +++++++------- .../q18a.sf100/simplified.txt | 358 ++++----- .../approved-plans-v2_7/q35.sf100/explain.txt | 416 +++++----- .../q35.sf100/simplified.txt | 198 ++--- .../approved-plans-v2_7/q35/explain.txt | 296 ++++---- .../approved-plans-v2_7/q35/simplified.txt | 72 +- .../q35a.sf100/explain.txt | 4 +- .../q35a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q35a/explain.txt | 4 +- .../approved-plans-v2_7/q35a/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 +- .../q70a.sf100/explain.txt | 4 +- .../q70a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 4 +- .../approved-plans-v2_7/q70a/simplified.txt | 2 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 282 +++---- .../q74.sf100/simplified.txt | 112 +-- .../approved-plans-v2_7/q74/explain.txt | 254 ++++--- .../approved-plans-v2_7/q74/simplified.txt | 96 +-- .../tpch-plan-stability/q18/explain.txt | 61 +- .../tpch-plan-stability/q18/simplified.txt | 19 +- .../tpch-plan-stability/q19/explain.txt | 18 +- .../tpch-plan-stability/q20/explain.txt | 119 ++- .../tpch-plan-stability/q20/simplified.txt | 29 +- .../tpch-plan-stability/q21/explain.txt | 4 +- .../tpch-plan-stability/q21/simplified.txt | 2 +- 135 files changed, 8206 insertions(+), 8502 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 25bf7e3dde34a..88081e501b834 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -1,63 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * SortMergeJoin Inner (47) - :- * Sort (41) - : +- Exchange (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (32) - : : +- * Filter (31) - : : +- * SortMergeJoin ExistenceJoin(exists#1) (30) - : : :- * SortMergeJoin ExistenceJoin(exists#2) (22) - : : : :- * SortMergeJoin LeftSemi (14) - : : : : :- * Sort (5) - : : : : : +- Exchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (13) - : : : : +- Exchange (12) - : : : : +- * Project (11) - : : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (9) - : : : +- * Sort (21) - : : : +- Exchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet spark_catalog.default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet spark_catalog.default.customer_address (33) - +- * Sort (46) - +- Exchange (45) - +- * Filter (44) - +- * ColumnarToRow (43) - +- Scan parquet spark_catalog.default.customer_demographics (42) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * Project (47) + +- * Filter (46) + +- * SortMergeJoin ExistenceJoin(exists#1) (45) + :- * SortMergeJoin ExistenceJoin(exists#2) (37) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildLeft (26) + : : :- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * SortMergeJoin LeftSemi (14) + : : : : :- * Sort (5) + : : : : : +- Exchange (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.customer (1) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- BroadcastExchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet spark_catalog.default.customer_address (15) + : : +- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet spark_catalog.default.customer_demographics (23) + : +- * Sort (36) + : +- Exchange (35) + : +- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * ColumnarToRow (31) + : : +- Scan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (32) + +- * Sort (44) + +- Exchange (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * ColumnarToRow (39) + : +- Scan parquet spark_catalog.default.catalog_sales (38) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -65,7 +64,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) AND might_contain(Subquery scalar-subquery#6, [id=#7], xxhash64(c_current_addr_sk#5, 42))) +Condition : (((isnotnull(c_customer_sk#3) AND isnotnull(c_current_addr_sk#5)) AND isnotnull(c_current_cdemo_sk#4)) AND might_contain(Subquery scalar-subquery#6, [id=#7], xxhash64(c_current_addr_sk#5, 42))) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -90,7 +89,7 @@ Input [2]: [ss_customer_sk#8, ss_sold_date_sk#9] Input [2]: [ss_customer_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#8) -(9) ReusedExchange [Reuses operator id: 64] +(9) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#11] (10) BroadcastHashJoin [codegen id : 4] @@ -111,268 +110,264 @@ Arguments: hashpartitioning(ss_customer_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id= Input [1]: [ss_customer_sk#8] Arguments: [ss_customer_sk#8 ASC NULLS FIRST], false, 0 -(14) SortMergeJoin [codegen id : 6] +(14) SortMergeJoin [codegen id : 7] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#8] Join type: LeftSemi Join condition: None -(15) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +(15) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#12, ca_county#13] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#10)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] +ReadSchema: struct -(16) ColumnarToRow [codegen id : 8] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +(16) ColumnarToRow [codegen id : 6] +Input [2]: [ca_address_sk#12, ca_county#13] -(17) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#14] +(17) Filter [codegen id : 6] +Input [2]: [ca_address_sk#12, ca_county#13] +Condition : (ca_county#13 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#12)) -(18) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(18) Project [codegen id : 6] +Output [1]: [ca_address_sk#12] +Input [2]: [ca_address_sk#12, ca_county#13] -(19) Project [codegen id : 8] -Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +(19) BroadcastExchange +Input [1]: [ca_address_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(20) Exchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: hashpartitioning(ws_bill_customer_sk#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#12] +Join type: Inner +Join condition: None -(21) Sort [codegen id : 9] -Input [1]: [ws_bill_customer_sk#12] -Arguments: [ws_bill_customer_sk#12 ASC NULLS FIRST], false, 0 +(21) Project [codegen id : 7] +Output [2]: [c_customer_sk#3, c_current_cdemo_sk#4] +Input [4]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#12] -(22) SortMergeJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None +(22) BroadcastExchange +Input [2]: [c_customer_sk#3, c_current_cdemo_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(23) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +(23) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(24) ColumnarToRow [codegen id : 12] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +(24) ColumnarToRow +Input [9]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -(25) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#17] +(25) Filter +Input [9]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] +Condition : isnotnull(cd_demo_sk#14) -(26) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +(26) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#14] Join type: Inner Join condition: None -(27) Project [codegen id : 12] -Output [1]: [cs_ship_customer_sk#15] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#17] +(27) Project [codegen id : 8] +Output [9]: [c_customer_sk#3, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] +Input [11]: [c_customer_sk#3, c_current_cdemo_sk#4, cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] (28) Exchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: hashpartitioning(cs_ship_customer_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(29) Sort [codegen id : 13] -Input [1]: [cs_ship_customer_sk#15] -Arguments: [cs_ship_customer_sk#15 ASC NULLS FIRST], false, 0 +Input [9]: [c_customer_sk#3, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] +Arguments: hashpartitioning(c_customer_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(30) SortMergeJoin [codegen id : 15] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(31) Filter [codegen id : 15] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(32) Project [codegen id : 15] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] +(29) Sort [codegen id : 9] +Input [9]: [c_customer_sk#3, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] +Arguments: [c_customer_sk#3 ASC NULLS FIRST], false, 0 -(33) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(30) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#18, ca_county#19] - -(35) Filter [codegen id : 14] -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +ReadSchema: struct -(36) Project [codegen id : 14] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_county#19] +(31) ColumnarToRow [codegen id : 11] +Input [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] -(37) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(32) ReusedExchange [Reuses operator id: 63] +Output [1]: [d_date_sk#25] -(38) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +(33) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#25] Join type: Inner Join condition: None -(39) Project [codegen id : 15] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] +(34) Project [codegen id : 11] +Output [1]: [ws_bill_customer_sk#23] +Input [3]: [ws_bill_customer_sk#23, ws_sold_date_sk#24, d_date_sk#25] -(40) Exchange -Input [1]: [c_current_cdemo_sk#4] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(35) Exchange +Input [1]: [ws_bill_customer_sk#23] +Arguments: hashpartitioning(ws_bill_customer_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(41) Sort [codegen id : 16] -Input [1]: [c_current_cdemo_sk#4] -Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 12] +Input [1]: [ws_bill_customer_sk#23] +Arguments: [ws_bill_customer_sk#23 ASC NULLS FIRST], false, 0 -(42) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) SortMergeJoin [codegen id : 13] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ws_bill_customer_sk#23] +Join type: ExistenceJoin(exists#2) +Join condition: None + +(38) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#26, cs_sold_date_sk#27] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(cs_sold_date_sk#27 IN dynamicpruning#10)] +ReadSchema: struct -(43) ColumnarToRow [codegen id : 17] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(39) ColumnarToRow [codegen id : 15] +Input [2]: [cs_ship_customer_sk#26, cs_sold_date_sk#27] -(44) Filter [codegen id : 17] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) +(40) ReusedExchange [Reuses operator id: 63] +Output [1]: [d_date_sk#28] -(45) Exchange -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: hashpartitioning(cd_demo_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(41) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#28] +Join type: Inner +Join condition: None -(46) Sort [codegen id : 18] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#20 ASC NULLS FIRST], false, 0 +(42) Project [codegen id : 15] +Output [1]: [cs_ship_customer_sk#26] +Input [3]: [cs_ship_customer_sk#26, cs_sold_date_sk#27, d_date_sk#28] -(47) SortMergeJoin [codegen id : 19] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner +(43) Exchange +Input [1]: [cs_ship_customer_sk#26] +Arguments: hashpartitioning(cs_ship_customer_sk#26, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(44) Sort [codegen id : 16] +Input [1]: [cs_ship_customer_sk#26] +Arguments: [cs_ship_customer_sk#26 ASC NULLS FIRST], false, 0 + +(45) SortMergeJoin [codegen id : 17] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#26] +Join type: ExistenceJoin(exists#1) Join condition: None -(48) Project [codegen id : 19] -Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(46) Filter [codegen id : 17] +Input [11]: [c_customer_sk#3, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, exists#2, exists#1] +Condition : (exists#2 OR exists#1) + +(47) Project [codegen id : 17] +Output [8]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] +Input [11]: [c_customer_sk#3, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, exists#2, exists#1] -(49) HashAggregate [codegen id : 19] -Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(48) HashAggregate [codegen id : 17] +Input [8]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] +Keys [8]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] -Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Results [9]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#30] -(50) Exchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(49) Exchange +Input [9]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#30] +Arguments: hashpartitioning(cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(51) HashAggregate [codegen id : 20] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(50) HashAggregate [codegen id : 18] +Input [9]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#30] +Keys [8]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] +Results [14]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, count(1)#31 AS cnt1#32, cd_purchase_estimate#18, count(1)#31 AS cnt2#33, cd_credit_rating#19, count(1)#31 AS cnt3#34, cd_dep_count#20, count(1)#31 AS cnt4#35, cd_dep_employed_count#21, count(1)#31 AS cnt5#36, cd_dep_college_count#22, count(1)#31 AS cnt6#37] -(52) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(51) TakeOrderedAndProject +Input [14]: [cd_gender#15, cd_marital_status#16, cd_education_status#17, cnt1#32, cd_purchase_estimate#18, cnt2#33, cd_credit_rating#19, cnt3#34, cd_dep_count#20, cnt4#35, cd_dep_employed_count#21, cnt5#36, cd_dep_college_count#22, cnt6#37] +Arguments: 100, [cd_gender#15 ASC NULLS FIRST, cd_marital_status#16 ASC NULLS FIRST, cd_education_status#17 ASC NULLS FIRST, cd_purchase_estimate#18 ASC NULLS FIRST, cd_credit_rating#19 ASC NULLS FIRST, cd_dep_count#20 ASC NULLS FIRST, cd_dep_employed_count#21 ASC NULLS FIRST, cd_dep_college_count#22 ASC NULLS FIRST], [cd_gender#15, cd_marital_status#16, cd_education_status#17, cnt1#32, cd_purchase_estimate#18, cnt2#33, cd_credit_rating#19, cnt3#34, cd_dep_count#20, cnt4#35, cd_dep_employed_count#21, cnt5#36, cd_dep_college_count#22, cnt6#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#6, [id=#7] -ObjectHashAggregate (59) -+- Exchange (58) - +- ObjectHashAggregate (57) - +- * Project (56) - +- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet spark_catalog.default.customer_address (53) +ObjectHashAggregate (58) ++- Exchange (57) + +- ObjectHashAggregate (56) + +- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet spark_catalog.default.customer_address (52) -(53) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(52) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#12, ca_county#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] -Input [2]: [ca_address_sk#18, ca_county#19] +(53) ColumnarToRow [codegen id : 1] +Input [2]: [ca_address_sk#12, ca_county#13] -(55) Filter [codegen id : 1] -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) +(54) Filter [codegen id : 1] +Input [2]: [ca_address_sk#12, ca_county#13] +Condition : (ca_county#13 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#12)) -(56) Project [codegen id : 1] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_county#19] +(55) Project [codegen id : 1] +Output [1]: [ca_address_sk#12] +Input [2]: [ca_address_sk#12, ca_county#13] -(57) ObjectHashAggregate -Input [1]: [ca_address_sk#18] +(56) ObjectHashAggregate +Input [1]: [ca_address_sk#12] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)] +Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#12, 42), 2555, 57765, 0, 0)] Aggregate Attributes [1]: [buf#38] Results [1]: [buf#39] -(58) Exchange +(57) Exchange Input [1]: [buf#39] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(59) ObjectHashAggregate +(58) ObjectHashAggregate Input [1]: [buf#39] Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)#40] -Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#18, 42), 2555, 57765, 0, 0)#40 AS bloomFilter#41] +Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#12, 42), 2555, 57765, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#12, 42), 2555, 57765, 0, 0)#40] +Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#12, 42), 2555, 57765, 0, 0)#40 AS bloomFilter#41] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (64) -+- * Project (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet spark_catalog.default.date_dim (60) +BroadcastExchange (63) ++- * Project (62) + +- * Filter (61) + +- * ColumnarToRow (60) + +- Scan parquet spark_catalog.default.date_dim (59) -(60) Scan parquet spark_catalog.default.date_dim +(59) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#42, d_moy#43] -(62) Filter [codegen id : 1] +(61) Filter [codegen id : 1] Input [3]: [d_date_sk#11, d_year#42, d_moy#43] Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 1)) AND (d_moy#43 <= 4)) AND isnotnull(d_date_sk#11)) -(63) Project [codegen id : 1] +(62) Project [codegen id : 1] Output [1]: [d_date_sk#11] Input [3]: [d_date_sk#11, d_year#42, d_moy#43] -(64) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index 50e9619e3430e..76b02d539fe1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -1,114 +1,109 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (20) + WholeStageCodegen (18) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (19) + WholeStageCodegen (17) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (16) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (15) - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - SortMergeJoin [c_customer_sk,cs_ship_customer_sk] - InputAdapter - WholeStageCodegen (10) - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] - InputAdapter - WholeStageCodegen (6) - SortMergeJoin [c_customer_sk,ss_customer_sk] - InputAdapter - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #3 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Subquery #1 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 2555, 57765, 0, 0),bloomFilter,buf] - Exchange #4 - ObjectHashAggregate [ca_address_sk] [buf,buf] - WholeStageCodegen (1) - Project [ca_address_sk] - Filter [ca_county,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + SortMergeJoin [c_customer_sk,cs_ship_customer_sk] + InputAdapter + WholeStageCodegen (13) + SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #2 + WholeStageCodegen (8) + Project [c_customer_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (7) + Project [c_customer_sk,c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #4 + WholeStageCodegen (1) + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] + Subquery #1 + ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 2555, 57765, 0, 0),bloomFilter,buf] + Exchange #5 + ObjectHashAggregate [ca_address_sk] [buf,buf] + WholeStageCodegen (1) + Project [ca_address_sk] + Filter [ca_county,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #6 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #5 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (9) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #7 - WholeStageCodegen (8) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastExchange #8 + WholeStageCodegen (6) + Project [ca_address_sk] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (13) - Sort [cs_ship_customer_sk] - InputAdapter - Exchange [cs_ship_customer_sk] #8 - WholeStageCodegen (12) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - Project [ca_address_sk] - Filter [ca_county,ca_address_sk] + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - WholeStageCodegen (18) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #10 - WholeStageCodegen (17) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + InputAdapter + WholeStageCodegen (12) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #9 + WholeStageCodegen (11) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + WholeStageCodegen (16) + Sort [cs_ship_customer_sk] + InputAdapter + Exchange [cs_ship_customer_sk] #10 + WholeStageCodegen (15) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 123f5c7ae3eed..4ad9a171a00b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -4,52 +4,52 @@ TakeOrderedAndProject (44) +- Exchange (42) +- * HashAggregate (41) +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * Filter (26) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (25) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (18) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (7) - : : : +- BroadcastExchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * ColumnarToRow (13) - : : : : +- Scan parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (38) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet spark_catalog.default.customer_demographics (35) + +- * Filter (39) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (38) + :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (31) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet spark_catalog.default.customer_address (12) + : : +- BroadcastExchange (22) + : : +- * Filter (21) + : : +- * ColumnarToRow (20) + : : +- Scan parquet spark_catalog.default.customer_demographics (19) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * ColumnarToRow (26) + : : +- Scan parquet spark_catalog.default.web_sales (25) + : +- ReusedExchange (27) + +- BroadcastExchange (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * ColumnarToRow (33) + : +- Scan parquet spark_catalog.default.catalog_sales (32) + +- ReusedExchange (34) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -57,7 +57,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : ((isnotnull(c_customer_sk#3) AND isnotnull(c_current_addr_sk#5)) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] @@ -97,161 +97,161 @@ Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(12) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(12) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#10, ca_county#11] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(13) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#10, ca_county#11] -(13) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) Filter [codegen id : 3] +Input [2]: [ca_address_sk#10, ca_county#11] +Condition : (ca_county#11 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#10)) -(14) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#12] +(15) Project [codegen id : 3] +Output [1]: [ca_address_sk#10] +Input [2]: [ca_address_sk#10, ca_county#11] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] +(16) BroadcastExchange +Input [1]: [ca_address_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] +(18) Project [codegen id : 9] +Output [2]: [c_customer_sk#3, c_current_cdemo_sk#4] +Input [4]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#10] -(17) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) +(20) ColumnarToRow [codegen id : 4] +Input [9]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] + +(21) Filter [codegen id : 4] +Input [9]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] +Condition : isnotnull(cd_demo_sk#12) + +(22) BroadcastExchange +Input [9]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#12] +Join type: Inner Join condition: None -(19) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(24) Project [codegen id : 9] +Output [9]: [c_customer_sk#3, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] +Input [11]: [c_customer_sk#3, c_current_cdemo_sk#4, cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] + +(25) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#21, ws_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct +PartitionFilters: [isnotnull(ws_sold_date_sk#22), dynamicpruningexpression(ws_sold_date_sk#22 IN dynamicpruning#8)] +ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(26) ColumnarToRow [codegen id : 6] +Input [2]: [ws_bill_customer_sk#21, ws_sold_date_sk#22] -(21) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#15] +(27) ReusedExchange [Reuses operator id: 49] +Output [1]: [d_date_sk#23] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#22] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(23) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +(29) Project [codegen id : 6] +Output [1]: [ws_bill_customer_sk#21] +Input [3]: [ws_bill_customer_sk#21, ws_sold_date_sk#22, d_date_sk#23] -(24) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(30) BroadcastExchange +Input [1]: [ws_bill_customer_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) +Right keys [1]: [ws_bill_customer_sk#21] +Join type: ExistenceJoin(exists#2) Join condition: None -(26) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(27) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(28) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_county#17] +(32) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#24, cs_sold_date_sk#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_county#17] - -(30) Filter [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_county#17] -Condition : (ca_county#17 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#16)) +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#25), dynamicpruningexpression(cs_sold_date_sk#25 IN dynamicpruning#8)] +ReadSchema: struct -(31) Project [codegen id : 7] -Output [1]: [ca_address_sk#16] -Input [2]: [ca_address_sk#16, ca_county#17] +(33) ColumnarToRow [codegen id : 8] +Input [2]: [cs_ship_customer_sk#24, cs_sold_date_sk#25] -(32) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(34) ReusedExchange [Reuses operator id: 49] +Output [1]: [d_date_sk#26] -(33) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#25] +Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(34) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16] +(36) Project [codegen id : 8] +Output [1]: [cs_ship_customer_sk#24] +Input [3]: [cs_ship_customer_sk#24, cs_sold_date_sk#25, d_date_sk#26] -(35) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(37) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#18) - -(38) BroadcastExchange -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(37) BroadcastExchange +Input [1]: [cs_ship_customer_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner +(38) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#24] +Join type: ExistenceJoin(exists#1) Join condition: None +(39) Filter [codegen id : 9] +Input [11]: [c_customer_sk#3, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20, exists#2, exists#1] +Condition : (exists#2 OR exists#1) + (40) Project [codegen id : 9] -Output [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Output [8]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] +Input [11]: [c_customer_sk#3, cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20, exists#2, exists#1] (41) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Input [8]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] +Keys [8]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#27] -Results [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] +Results [9]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20, count#28] (42) Exchange -Input [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] -Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [9]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20, count#28] +Arguments: hashpartitioning(cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20, 5), ENSURE_REQUIREMENTS, [plan_id=6] (43) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#28] -Keys [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Input [9]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20, count#28] +Keys [8]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cd_purchase_estimate#16, cd_credit_rating#17, cd_dep_count#18, cd_dep_employed_count#19, cd_dep_college_count#20] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#29] -Results [14]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#29 AS cnt1#30, cd_purchase_estimate#22, count(1)#29 AS cnt2#31, cd_credit_rating#23, count(1)#29 AS cnt3#32, cd_dep_count#24, count(1)#29 AS cnt4#33, cd_dep_employed_count#25, count(1)#29 AS cnt5#34, cd_dep_college_count#26, count(1)#29 AS cnt6#35] +Results [14]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, count(1)#29 AS cnt1#30, cd_purchase_estimate#16, count(1)#29 AS cnt2#31, cd_credit_rating#17, count(1)#29 AS cnt3#32, cd_dep_count#18, count(1)#29 AS cnt4#33, cd_dep_employed_count#19, count(1)#29 AS cnt5#34, cd_dep_college_count#20, count(1)#29 AS cnt6#35] (44) TakeOrderedAndProject -Input [14]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#30, cd_purchase_estimate#22, cnt2#31, cd_credit_rating#23, cnt3#32, cd_dep_count#24, cnt4#33, cd_dep_employed_count#25, cnt5#34, cd_dep_college_count#26, cnt6#35] -Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#30, cd_purchase_estimate#22, cnt2#31, cd_credit_rating#23, cnt3#32, cd_dep_count#24, cnt4#33, cd_dep_employed_count#25, cnt5#34, cd_dep_college_count#26, cnt6#35] +Input [14]: [cd_gender#13, cd_marital_status#14, cd_education_status#15, cnt1#30, cd_purchase_estimate#16, cnt2#31, cd_credit_rating#17, cnt3#32, cd_dep_count#18, cnt4#33, cd_dep_employed_count#19, cnt5#34, cd_dep_college_count#20, cnt6#35] +Arguments: 100, [cd_gender#13 ASC NULLS FIRST, cd_marital_status#14 ASC NULLS FIRST, cd_education_status#15 ASC NULLS FIRST, cd_purchase_estimate#16 ASC NULLS FIRST, cd_credit_rating#17 ASC NULLS FIRST, cd_dep_count#18 ASC NULLS FIRST, cd_dep_employed_count#19 ASC NULLS FIRST, cd_dep_college_count#20 ASC NULLS FIRST], [cd_gender#13, cd_marital_status#14, cd_education_status#15, cnt1#30, cd_purchase_estimate#16, cnt2#31, cd_credit_rating#17, cnt3#32, cd_dep_count#18, cnt4#33, cd_dep_employed_count#19, cnt5#34, cd_dep_college_count#20, cnt6#35] ===== Subqueries ===== @@ -285,8 +285,8 @@ Input [3]: [d_date_sk#9, d_year#36, d_moy#37] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#22 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#25 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 33157630a88d2..e11836a971aea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (9) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_customer_sk,c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -39,38 +39,38 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + WholeStageCodegen (3) + Project [ca_address_sk] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - Project [ca_address_sk] - Filter [ca_county,ca_address_sk] + WholeStageCodegen (6) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index 5420d99d3e82c..09f406c1bb71c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (80) -+- * Project (79) - +- * SortMergeJoin Inner (78) - :- * Project (60) - : +- * SortMergeJoin Inner (59) +TakeOrderedAndProject (82) ++- * Project (81) + +- * SortMergeJoin Inner (80) + :- * Project (61) + : +- * SortMergeJoin Inner (60) : :- * Project (40) : : +- * SortMergeJoin Inner (39) : : :- * Sort (21) @@ -44,41 +44,43 @@ TakeOrderedAndProject (80) : : : +- ReusedExchange (25) : : +- * Sort (31) : : +- ReusedExchange (30) - : +- * Sort (58) - : +- Exchange (57) - : +- * Filter (56) - : +- * HashAggregate (55) - : +- Exchange (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * SortMergeJoin Inner (51) - : :- * Sort (48) - : : +- Exchange (47) - : : +- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Filter (43) - : : : +- * ColumnarToRow (42) - : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : +- ReusedExchange (44) - : +- * Sort (50) - : +- ReusedExchange (49) - +- * Sort (77) - +- Exchange (76) - +- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * SortMergeJoin Inner (71) - :- * Sort (68) - : +- Exchange (67) - : +- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Filter (63) - : : +- * ColumnarToRow (62) - : : +- Scan parquet spark_catalog.default.web_sales (61) - : +- ReusedExchange (64) - +- * Sort (70) - +- ReusedExchange (69) + : +- * Sort (59) + : +- Exchange (58) + : +- * Project (57) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- Exchange (54) + : +- * HashAggregate (53) + : +- * Project (52) + : +- * SortMergeJoin Inner (51) + : :- * Sort (48) + : : +- Exchange (47) + : : +- * Project (46) + : : +- * BroadcastHashJoin Inner BuildRight (45) + : : :- * Filter (43) + : : : +- * ColumnarToRow (42) + : : : +- Scan parquet spark_catalog.default.web_sales (41) + : : +- ReusedExchange (44) + : +- * Sort (50) + : +- ReusedExchange (49) + +- * Sort (79) + +- Exchange (78) + +- * Project (77) + +- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * SortMergeJoin Inner (72) + :- * Sort (69) + : +- Exchange (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Filter (64) + : : +- * ColumnarToRow (63) + : : +- Scan parquet spark_catalog.default.web_sales (62) + : +- ReusedExchange (65) + +- * Sort (71) + +- ReusedExchange (70) (1) Scan parquet spark_catalog.default.store_sales @@ -96,7 +98,7 @@ Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sol Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 84] +(4) ReusedExchange [Reuses operator id: 86] Output [2]: [d_date_sk#6, d_year#7] (5) BroadcastHashJoin [codegen id : 2] @@ -194,7 +196,7 @@ Input [4]: [ss_customer_sk#21, ss_ext_discount_amt#22, ss_ext_list_price#23, ss_ Input [4]: [ss_customer_sk#21, ss_ext_discount_amt#22, ss_ext_list_price#23, ss_sold_date_sk#24] Condition : isnotnull(ss_customer_sk#21) -(25) ReusedExchange [Reuses operator id: 88] +(25) ReusedExchange [Reuses operator id: 90] Output [2]: [d_date_sk#26, d_year#27] (26) BroadcastHashJoin [codegen id : 10] @@ -283,7 +285,7 @@ Input [4]: [ws_bill_customer_sk#41, ws_ext_discount_amt#42, ws_ext_list_price#43 Input [4]: [ws_bill_customer_sk#41, ws_ext_discount_amt#42, ws_ext_list_price#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#41) -(44) ReusedExchange [Reuses operator id: 84] +(44) ReusedExchange [Reuses operator id: 86] Output [2]: [d_date_sk#45, d_year#46] (45) BroadcastHashJoin [codegen id : 19] @@ -343,171 +345,179 @@ Results [2]: [c_customer_id#48 AS customer_id#58, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#58, year_total#59] Condition : (isnotnull(year_total#59) AND (year_total#59 > 0.00)) -(57) Exchange +(57) Project [codegen id : 24] +Output [2]: [customer_id#58 AS customer_id#60, year_total#59 AS year_total#61] Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(58) Sort [codegen id : 25] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +(58) Exchange +Input [2]: [customer_id#60, year_total#61] +Arguments: hashpartitioning(customer_id#60, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(59) Sort [codegen id : 25] +Input [2]: [customer_id#60, year_total#61] +Arguments: [customer_id#60 ASC NULLS FIRST], false, 0 -(59) SortMergeJoin [codegen id : 26] +(60) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#58] +Right keys [1]: [customer_id#60] Join type: Inner Join condition: None -(60) Project [codegen id : 26] -Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#59] -Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#58, year_total#59] +(61) Project [codegen id : 26] +Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#61] +Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#60, year_total#61] -(61) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +(62) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +(63) ColumnarToRow [codegen id : 28] +Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65] -(63) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) +(64) Filter [codegen id : 28] +Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#62) -(64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#64, d_year#65] +(65) ReusedExchange [Reuses operator id: 90] +Output [2]: [d_date_sk#66, d_year#67] -(65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] +(66) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(66) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65] -Input [6]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] +(67) Project [codegen id : 28] +Output [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] +Input [6]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65, d_date_sk#66, d_year#67] -(67) Exchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65] -Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(68) Exchange +Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] +Arguments: hashpartitioning(ws_bill_customer_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(68) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65] -Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 29] +Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] +Arguments: [ws_bill_customer_sk#62 ASC NULLS FIRST], false, 0 -(69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] +(70) ReusedExchange [Reuses operator id: 12] +Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -(70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] -Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 +(71) Sort [codegen id : 31] +Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] +Arguments: [c_customer_sk#68 ASC NULLS FIRST], false, 0 -(71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#66] +(72) SortMergeJoin [codegen id : 32] +Left keys [1]: [ws_bill_customer_sk#62] +Right keys [1]: [c_customer_sk#68] Join type: Inner Join condition: None -(72) Project [codegen id : 32] -Output [10]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65] -Input [12]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65, c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] - -(73) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65] -Keys [8]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] -Aggregate Attributes [1]: [sum#74] -Results [9]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65, sum#75] - -(74) Exchange -Input [9]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65, sum#75] -Arguments: hashpartitioning(c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(75) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65, sum#75] -Keys [8]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))#57] -Results [2]: [c_customer_id#67 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))#57,18,2) AS year_total#77] - -(76) Exchange -Input [2]: [customer_id#76, year_total#77] -Arguments: hashpartitioning(customer_id#76, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(77) Sort [codegen id : 34] -Input [2]: [customer_id#76, year_total#77] -Arguments: [customer_id#76 ASC NULLS FIRST], false, 0 - -(78) SortMergeJoin [codegen id : 35] +(73) Project [codegen id : 32] +Output [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] +Input [12]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67, c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] + +(74) HashAggregate [codegen id : 32] +Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] +Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))] +Aggregate Attributes [1]: [sum#76] +Results [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77] + +(75) Exchange +Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77] +Arguments: hashpartitioning(c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(76) HashAggregate [codegen id : 33] +Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77] +Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))#57] +Results [2]: [c_customer_id#69 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))#57,18,2) AS year_total#79] + +(77) Project [codegen id : 33] +Output [2]: [customer_id#78 AS customer_id#80, year_total#79 AS year_total#81] +Input [2]: [customer_id#78, year_total#79] + +(78) Exchange +Input [2]: [customer_id#80, year_total#81] +Arguments: hashpartitioning(customer_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(79) Sort [codegen id : 34] +Input [2]: [customer_id#80, year_total#81] +Arguments: [customer_id#80 ASC NULLS FIRST], false, 0 + +(80) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#76] +Right keys [1]: [customer_id#80] Join type: Inner -Join condition: (CASE WHEN (year_total#59 > 0.00) THEN (year_total#77 / year_total#59) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) +Join condition: (CASE WHEN (year_total#61 > 0.00) THEN (year_total#81 / year_total#61) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) -(79) Project [codegen id : 35] +(81) Project [codegen id : 35] Output [1]: [customer_preferred_cust_flag#39] -Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#59, customer_id#76, year_total#77] +Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#61, customer_id#80, year_total#81] -(80) TakeOrderedAndProject +(82) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#39] Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (84) -+- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet spark_catalog.default.date_dim (81) +BroadcastExchange (86) ++- * Filter (85) + +- * ColumnarToRow (84) + +- Scan parquet spark_catalog.default.date_dim (83) -(81) Scan parquet spark_catalog.default.date_dim +(83) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 1] +(84) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#6, d_year#7] -(83) Filter [codegen id : 1] +(85) Filter [codegen id : 1] Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk#6)) -(84) BroadcastExchange +(86) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (88) -+- * Filter (87) - +- * ColumnarToRow (86) - +- Scan parquet spark_catalog.default.date_dim (85) +BroadcastExchange (90) ++- * Filter (89) + +- * ColumnarToRow (88) + +- Scan parquet spark_catalog.default.date_dim (87) -(85) Scan parquet spark_catalog.default.date_dim +(87) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#26, d_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 1] +(88) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#26, d_year#27] -(87) Filter [codegen id : 1] +(89) Filter [codegen id : 1] Input [2]: [d_date_sk#26, d_year#27] Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2002)) AND isnotnull(d_date_sk#26)) -(88) BroadcastExchange +(90) BroadcastExchange Input [2]: [d_date_sk#26, d_year#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#25 +Subquery:4 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt index af7795400c38f..54339c64f2b7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt @@ -100,64 +100,66 @@ TakeOrderedAndProject [customer_preferred_cust_flag] InputAdapter Exchange [customer_id] #10 WholeStageCodegen (24) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (20) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (19) - Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #4 - InputAdapter - WholeStageCodegen (22) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (23) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - WholeStageCodegen (32) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (28) - Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #9 - InputAdapter - WholeStageCodegen (31) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (32) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (28) + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #9 + InputAdapter + WholeStageCodegen (31) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 560436cd42605..f5857954a0bd0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) +TakeOrderedAndProject (74) ++- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) : :- * Project (34) : : +- * BroadcastHashJoin Inner BuildRight (33) : : :- * Filter (16) @@ -38,39 +38,41 @@ TakeOrderedAndProject (72) : : : +- * ColumnarToRow (21) : : : +- Scan parquet spark_catalog.default.store_sales (20) : : +- ReusedExchange (26) - : +- BroadcastExchange (51) - : +- * Filter (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * HashAggregate (47) - : +- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (43) - : : +- * BroadcastHashJoin Inner BuildRight (42) - : : :- * Filter (37) - : : : +- * ColumnarToRow (36) - : : : +- Scan parquet spark_catalog.default.customer (35) - : : +- BroadcastExchange (41) - : : +- * Filter (40) - : : +- * ColumnarToRow (39) - : : +- Scan parquet spark_catalog.default.web_sales (38) - : +- ReusedExchange (44) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Filter (56) - : : +- * ColumnarToRow (55) - : : +- Scan parquet spark_catalog.default.customer (54) - : +- BroadcastExchange (60) - : +- * Filter (59) - : +- * ColumnarToRow (58) - : +- Scan parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (63) + : +- BroadcastExchange (52) + : +- * Project (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * Filter (37) + : : : +- * ColumnarToRow (36) + : : : +- Scan parquet spark_catalog.default.customer (35) + : : +- BroadcastExchange (41) + : : +- * Filter (40) + : : +- * ColumnarToRow (39) + : : +- Scan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (44) + +- BroadcastExchange (71) + +- * Project (70) + +- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- * Project (63) + : +- * BroadcastHashJoin Inner BuildRight (62) + : :- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet spark_catalog.default.customer (55) + : +- BroadcastExchange (61) + : +- * Filter (60) + : +- * ColumnarToRow (59) + : +- Scan parquet spark_catalog.default.web_sales (58) + +- ReusedExchange (64) (1) Scan parquet spark_catalog.default.customer @@ -116,7 +118,7 @@ Join condition: None Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 76] +(10) ReusedExchange [Reuses operator id: 78] Output [2]: [d_date_sk#14, d_year#15] (11) BroadcastHashJoin [codegen id : 3] @@ -194,7 +196,7 @@ Join condition: None Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(26) ReusedExchange [Reuses operator id: 80] +(26) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#34, d_year#35] (27) BroadcastHashJoin [codegen id : 6] @@ -282,7 +284,7 @@ Join condition: None Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(44) ReusedExchange [Reuses operator id: 76] +(44) ReusedExchange [Reuses operator id: 78] Output [2]: [d_date_sk#53, d_year#54] (45) BroadcastHashJoin [codegen id : 10] @@ -317,166 +319,174 @@ Results [2]: [c_customer_id#42 AS customer_id#58, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#58, year_total#59] Condition : (isnotnull(year_total#59) AND (year_total#59 > 0.00)) -(51) BroadcastExchange +(51) Project [codegen id : 11] +Output [2]: [customer_id#58 AS customer_id#60, year_total#59 AS year_total#61] Input [2]: [customer_id#58, year_total#59] + +(52) BroadcastExchange +Input [2]: [customer_id#60, year_total#61] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(52) BroadcastHashJoin [codegen id : 16] +(53) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#58] +Right keys [1]: [customer_id#60] Join type: Inner Join condition: None -(53) Project [codegen id : 16] -Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#59] -Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#58, year_total#59] +(54) Project [codegen id : 16] +Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#61] +Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#60, year_total#61] -(54) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] +(55) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] +(56) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] -(56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Condition : (isnotnull(c_customer_sk#60) AND isnotnull(c_customer_id#61)) +(57) Filter [codegen id : 14] +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) -(57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +(58) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#71), dynamicpruningexpression(ws_sold_date_sk#71 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ws_sold_date_sk#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#33)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +(59) ColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] -(59) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] -Condition : isnotnull(ws_bill_customer_sk#68) +(60) Filter [codegen id : 12] +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +Condition : isnotnull(ws_bill_customer_sk#70) -(60) BroadcastExchange -Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +(61) BroadcastExchange +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#60] -Right keys [1]: [ws_bill_customer_sk#68] +(62) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#62] +Right keys [1]: [ws_bill_customer_sk#70] Join type: Inner Join condition: None -(62) Project [codegen id : 14] -Output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] -Input [12]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +(63) Project [codegen id : 14] +Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +Input [12]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] -(63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#72, d_year#73] +(64) ReusedExchange [Reuses operator id: 82] +Output [2]: [d_date_sk#74, d_year#75] -(64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#71] -Right keys [1]: [d_date_sk#72] +(65) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#73] +Right keys [1]: [d_date_sk#74] Join type: Inner Join condition: None -(65) Project [codegen id : 14] -Output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#73] -Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71, d_date_sk#72, d_year#73] - -(66) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#73] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))] -Aggregate Attributes [1]: [sum#74] -Results [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73, sum#75] - -(67) Exchange -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73, sum#75] -Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(68) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73, sum#75] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57] -Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57,18,2) AS year_total#77] - -(69) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] +(66) Project [codegen id : 14] +Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73, d_date_sk#74, d_year#75] + +(67) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))] +Aggregate Attributes [1]: [sum#76] +Results [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77] + +(68) Exchange +Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77] +Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(69) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))#57] +Results [2]: [c_customer_id#63 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))#57,18,2) AS year_total#79] + +(70) Project [codegen id : 15] +Output [2]: [customer_id#78 AS customer_id#80, year_total#79 AS year_total#81] +Input [2]: [customer_id#78, year_total#79] + +(71) BroadcastExchange +Input [2]: [customer_id#80, year_total#81] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(70) BroadcastHashJoin [codegen id : 16] +(72) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#76] +Right keys [1]: [customer_id#80] Join type: Inner -Join condition: (CASE WHEN (year_total#59 > 0.00) THEN (year_total#77 / year_total#59) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) +Join condition: (CASE WHEN (year_total#61 > 0.00) THEN (year_total#81 / year_total#61) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) -(71) Project [codegen id : 16] +(73) Project [codegen id : 16] Output [1]: [customer_preferred_cust_flag#39] -Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#59, customer_id#76, year_total#77] +Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#61, customer_id#80, year_total#81] -(72) TakeOrderedAndProject +(74) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#39] Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (76) -+- * Filter (75) - +- * ColumnarToRow (74) - +- Scan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (78) ++- * Filter (77) + +- * ColumnarToRow (76) + +- Scan parquet spark_catalog.default.date_dim (75) -(73) Scan parquet spark_catalog.default.date_dim +(75) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(74) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(75) Filter [codegen id : 1] +(77) Filter [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(76) BroadcastExchange +(78) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (80) -+- * Filter (79) - +- * ColumnarToRow (78) - +- Scan parquet spark_catalog.default.date_dim (77) +BroadcastExchange (82) ++- * Filter (81) + +- * ColumnarToRow (80) + +- Scan parquet spark_catalog.default.date_dim (79) -(77) Scan parquet spark_catalog.default.date_dim +(79) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#34, d_year#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(78) ColumnarToRow [codegen id : 1] +(80) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(79) Filter [codegen id : 1] +(81) Filter [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) -(80) BroadcastExchange +(82) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#71 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#73 IN dynamicpruning#33 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index b490d34471caa..9c2523651d0ea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -71,53 +71,55 @@ TakeOrderedAndProject [customer_preferred_cust_flag] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 f069135dca882..d4e832116e90c 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 @@ -1,17 +1,17 @@ == Physical Plan == -TakeOrderedAndProject (111) -+- * HashAggregate (110) - +- Exchange (109) - +- * HashAggregate (108) - +- * Expand (107) - +- Union (106) - :- * Project (73) - : +- * Filter (72) - : +- * HashAggregate (71) - : +- Exchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) +TakeOrderedAndProject (106) ++- * HashAggregate (105) + +- Exchange (104) + +- * HashAggregate (103) + +- * Expand (102) + +- Union (101) + :- * Project (68) + : +- * Filter (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- * Project (63) + : +- * BroadcastHashJoin Inner BuildRight (62) : :- * Project (60) : : +- * BroadcastHashJoin Inner BuildRight (59) : : :- * BroadcastHashJoin LeftSemi BuildRight (57) @@ -72,44 +72,39 @@ TakeOrderedAndProject (111) : : : : +- ReusedExchange (44) : : : +- ReusedExchange (47) : : +- ReusedExchange (58) - : +- BroadcastExchange (66) - : +- * BroadcastHashJoin LeftSemi BuildRight (65) - : :- * Filter (63) - : : +- * ColumnarToRow (62) - : : +- Scan parquet spark_catalog.default.item (61) - : +- ReusedExchange (64) - :- * Project (89) - : +- * Filter (88) - : +- * HashAggregate (87) - : +- Exchange (86) - : +- * HashAggregate (85) - : +- * Project (84) - : +- * BroadcastHashJoin Inner BuildRight (83) - : :- * Project (81) - : : +- * BroadcastHashJoin Inner BuildRight (80) - : : :- * BroadcastHashJoin LeftSemi BuildRight (78) - : : : :- * Filter (76) - : : : : +- * ColumnarToRow (75) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (74) - : : : +- ReusedExchange (77) - : : +- ReusedExchange (79) - : +- ReusedExchange (82) - +- * Project (105) - +- * Filter (104) - +- * HashAggregate (103) - +- Exchange (102) - +- * HashAggregate (101) - +- * Project (100) - +- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * BroadcastHashJoin LeftSemi BuildRight (94) - : : :- * Filter (92) - : : : +- * ColumnarToRow (91) - : : : +- Scan parquet spark_catalog.default.web_sales (90) - : : +- ReusedExchange (93) - : +- ReusedExchange (95) - +- ReusedExchange (98) + : +- ReusedExchange (61) + :- * Project (84) + : +- * Filter (83) + : +- * HashAggregate (82) + : +- Exchange (81) + : +- * HashAggregate (80) + : +- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * Project (76) + : : +- * BroadcastHashJoin Inner BuildRight (75) + : : :- * BroadcastHashJoin LeftSemi BuildRight (73) + : : : :- * Filter (71) + : : : : +- * ColumnarToRow (70) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (74) + : +- ReusedExchange (77) + +- * Project (100) + +- * Filter (99) + +- * HashAggregate (98) + +- Exchange (97) + +- * HashAggregate (96) + +- * Project (95) + +- * BroadcastHashJoin Inner BuildRight (94) + :- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * BroadcastHashJoin LeftSemi BuildRight (89) + : : :- * Filter (87) + : : : +- * ColumnarToRow (86) + : : : +- Scan parquet spark_catalog.default.web_sales (85) + : : +- ReusedExchange (88) + : +- ReusedExchange (90) + +- ReusedExchange (93) (1) Scan parquet spark_catalog.default.store_sales @@ -120,10 +115,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 37] +(2) ColumnarToRow [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 37] +(3) Filter [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -156,7 +151,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 140] +(10) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -206,7 +201,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 140] +(21) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -316,7 +311,7 @@ Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 140] +(44) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#30] (45) BroadcastHashJoin [codegen id : 14] @@ -374,89 +369,65 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 37] +(57) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 135] +(58) ReusedExchange [Reuses operator id: 130] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 37] +(59) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 37] +(60) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] -(61) Scan parquet spark_catalog.default.item +(61) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) ColumnarToRow [codegen id : 36] -Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(63) Filter [codegen id : 36] -Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -Condition : isnotnull(i_item_sk#37) - -(64) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#35] - -(65) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [i_item_sk#37] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(66) BroadcastExchange -Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(67) BroadcastHashJoin [codegen id : 37] +(62) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(68) Project [codegen id : 37] +(63) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(69) HashAggregate [codegen id : 37] +(64) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(70) Exchange +(65) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(71) HashAggregate [codegen id : 38] +(66) HashAggregate [codegen id : 21] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50] -(72) Filter [codegen id : 38] +(67) Filter [codegen id : 21] Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50] Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(73) Project [codegen id : 38] +(68) Project [codegen id : 21] Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#38 AS i_brand_id#54, i_class_id#39 AS i_class_id#55, i_category_id#40 AS i_category_id#56] Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50] -(74) Scan parquet spark_catalog.default.catalog_sales +(69) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] @@ -464,75 +435,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 75] +(70) ColumnarToRow [codegen id : 41] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(76) Filter [codegen id : 75] +(71) Filter [codegen id : 41] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Condition : isnotnull(cs_item_sk#57) -(77) ReusedExchange [Reuses operator id: 56] +(72) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#35] -(78) BroadcastHashJoin [codegen id : 75] +(73) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(79) ReusedExchange [Reuses operator id: 135] +(74) ReusedExchange [Reuses operator id: 130] Output [1]: [d_date_sk#61] -(80) BroadcastHashJoin [codegen id : 75] +(75) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#61] Join type: Inner Join condition: None -(81) Project [codegen id : 75] +(76) Project [codegen id : 41] Output [3]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59] Input [5]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, d_date_sk#61] -(82) ReusedExchange [Reuses operator id: 66] +(77) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(83) BroadcastHashJoin [codegen id : 75] +(78) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#57] Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(84) Project [codegen id : 75] +(79) Project [codegen id : 41] Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Input [7]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(85) HashAggregate [codegen id : 75] +(80) HashAggregate [codegen id : 41] Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71] -(86) Exchange +(81) Exchange Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(87) HashAggregate [codegen id : 76] +(82) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73] Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75] -(88) Filter [codegen id : 76] +(83) Filter [codegen id : 42] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(89) Project [codegen id : 76] +(84) Project [codegen id : 42] Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#63, i_class_id#64, i_category_id#65] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75] -(90) Scan parquet spark_catalog.default.web_sales +(85) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] @@ -540,291 +511,291 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 113] +(86) ColumnarToRow [codegen id : 62] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -(92) Filter [codegen id : 113] +(87) Filter [codegen id : 62] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Condition : isnotnull(ws_item_sk#77) -(93) ReusedExchange [Reuses operator id: 56] +(88) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#35] -(94) BroadcastHashJoin [codegen id : 113] +(89) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#77] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(95) ReusedExchange [Reuses operator id: 135] +(90) ReusedExchange [Reuses operator id: 130] Output [1]: [d_date_sk#81] -(96) BroadcastHashJoin [codegen id : 113] +(91) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_sold_date_sk#80] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(97) Project [codegen id : 113] +(92) Project [codegen id : 62] Output [3]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79] Input [5]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, d_date_sk#81] -(98) ReusedExchange [Reuses operator id: 66] +(93) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(99) BroadcastHashJoin [codegen id : 113] +(94) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#77] Right keys [1]: [i_item_sk#82] Join type: Inner Join condition: None -(100) Project [codegen id : 113] +(95) Project [codegen id : 62] Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] Input [7]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(101) HashAggregate [codegen id : 113] +(96) HashAggregate [codegen id : 62] Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88] Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91] -(102) Exchange +(97) Exchange Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91] -Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(103) HashAggregate [codegen id : 114] +(98) HashAggregate [codegen id : 63] Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91] Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93] Results [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95] -(104) Filter [codegen id : 114] +(99) Filter [codegen id : 63] Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95] Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(105) Project [codegen id : 114] +(100) Project [codegen id : 63] Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#83, i_class_id#84, i_category_id#85] Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95] -(106) Union +(101) Union -(107) Expand [codegen id : 115] +(102) Expand [codegen id : 64] Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -(108) HashAggregate [codegen id : 115] +(103) HashAggregate [codegen id : 64] Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -(109) Exchange +(104) Exchange Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(110) HashAggregate [codegen id : 116] +(105) HashAggregate [codegen id : 65] Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [sum(sales#49), sum(number_sales#50)] Aggregate Attributes [2]: [sum(sales#49)#108, sum(number_sales#50)#109] Results [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales#49)#108 AS sum(sales)#110, sum(number_sales#50)#109 AS sum(number_sales)#111] -(111) TakeOrderedAndProject +(106) TakeOrderedAndProject Input [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_class_id#99 ASC NULLS FIRST, i_category_id#100 ASC NULLS FIRST], [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] ===== Subqueries ===== -Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* HashAggregate (130) -+- Exchange (129) - +- * HashAggregate (128) - +- Union (127) +Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* HashAggregate (125) ++- Exchange (124) + +- * HashAggregate (123) + +- Union (122) + :- * Project (111) + : +- * BroadcastHashJoin Inner BuildRight (110) + : :- * ColumnarToRow (108) + : : +- Scan parquet spark_catalog.default.store_sales (107) + : +- ReusedExchange (109) :- * Project (116) : +- * BroadcastHashJoin Inner BuildRight (115) : :- * ColumnarToRow (113) - : : +- Scan parquet spark_catalog.default.store_sales (112) + : : +- Scan parquet spark_catalog.default.catalog_sales (112) : +- ReusedExchange (114) - :- * Project (121) - : +- * BroadcastHashJoin Inner BuildRight (120) - : :- * ColumnarToRow (118) - : : +- Scan parquet spark_catalog.default.catalog_sales (117) - : +- ReusedExchange (119) - +- * Project (126) - +- * BroadcastHashJoin Inner BuildRight (125) - :- * ColumnarToRow (123) - : +- Scan parquet spark_catalog.default.web_sales (122) - +- ReusedExchange (124) - - -(112) Scan parquet spark_catalog.default.store_sales + +- * Project (121) + +- * BroadcastHashJoin Inner BuildRight (120) + :- * ColumnarToRow (118) + : +- Scan parquet spark_catalog.default.web_sales (117) + +- ReusedExchange (119) + + +(107) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)] ReadSchema: struct -(113) ColumnarToRow [codegen id : 2] +(108) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] -(114) ReusedExchange [Reuses operator id: 140] +(109) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#115] -(115) BroadcastHashJoin [codegen id : 2] +(110) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#114] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(116) Project [codegen id : 2] +(111) Project [codegen id : 2] Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117] Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115] -(117) Scan parquet spark_catalog.default.catalog_sales +(112) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 4] +(113) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] -(119) ReusedExchange [Reuses operator id: 140] +(114) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#121] -(120) BroadcastHashJoin [codegen id : 4] +(115) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#120] Right keys [1]: [d_date_sk#121] Join type: Inner Join condition: None -(121) Project [codegen id : 4] +(116) Project [codegen id : 4] Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123] Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121] -(122) Scan parquet spark_catalog.default.web_sales +(117) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)] ReadSchema: struct -(123) ColumnarToRow [codegen id : 6] +(118) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] -(124) ReusedExchange [Reuses operator id: 140] +(119) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#127] -(125) BroadcastHashJoin [codegen id : 6] +(120) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#126] Right keys [1]: [d_date_sk#127] Join type: Inner Join condition: None -(126) Project [codegen id : 6] +(121) Project [codegen id : 6] Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129] Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127] -(127) Union +(122) Union -(128) HashAggregate [codegen id : 7] +(123) HashAggregate [codegen id : 7] Input [2]: [quantity#116, list_price#117] Keys: [] Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] Aggregate Attributes [2]: [sum#130, count#131] Results [2]: [sum#132, count#133] -(129) Exchange +(124) Exchange Input [2]: [sum#132, count#133] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(130) HashAggregate [codegen id : 8] +(125) HashAggregate [codegen id : 8] Input [2]: [sum#132, count#133] Keys: [] Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] Aggregate Attributes [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134] Results [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134 AS average_sales#135] -Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 107 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 117 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 112 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 122 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 117 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (135) -+- * Project (134) - +- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet spark_catalog.default.date_dim (131) +BroadcastExchange (130) ++- * Project (129) + +- * Filter (128) + +- * ColumnarToRow (127) + +- Scan parquet spark_catalog.default.date_dim (126) -(131) Scan parquet spark_catalog.default.date_dim +(126) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#36, d_year#136, d_moy#137] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 1] +(127) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#36, d_year#136, d_moy#137] -(133) Filter [codegen id : 1] +(128) Filter [codegen id : 1] Input [3]: [d_date_sk#36, d_year#136, d_moy#137] Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 = 2001)) AND (d_moy#137 = 11)) AND isnotnull(d_date_sk#36)) -(134) Project [codegen id : 1] +(129) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [3]: [d_date_sk#36, d_year#136, d_moy#137] -(135) BroadcastExchange +(130) BroadcastExchange Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (140) -+- * Project (139) - +- * Filter (138) - +- * ColumnarToRow (137) - +- Scan parquet spark_catalog.default.date_dim (136) +BroadcastExchange (135) ++- * Project (134) + +- * Filter (133) + +- * ColumnarToRow (132) + +- Scan parquet spark_catalog.default.date_dim (131) -(136) Scan parquet spark_catalog.default.date_dim +(131) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#138] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#138] -(138) Filter [codegen id : 1] +(133) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#138] Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#13)) -(139) Project [codegen id : 1] +(134) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#138] -(140) BroadcastExchange +(135) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:10 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:11 Hosting operator id = 99 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:12 Hosting operator id = 90 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 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 6e37e751722db..cf2a9d9a0b282 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 @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (116) + WholeStageCodegen (65) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (115) + WholeStageCodegen (64) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (38) + WholeStageCodegen (21) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 WholeStageCodegen (8) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #15 + Exchange #14 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (37) + WholeStageCodegen (20) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -171,23 +171,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (36) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 - WholeStageCodegen (76) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (42) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (75) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (41) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -204,15 +196,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (114) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (63) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (113) + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (62) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -229,4 +221,4 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 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 100a723e381c2..b187a11721080 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 @@ -1,19 +1,19 @@ == Physical Plan == -TakeOrderedAndProject (105) -+- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * Project (67) - : +- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) +TakeOrderedAndProject (100) ++- * HashAggregate (99) + +- Exchange (98) + +- * HashAggregate (97) + +- * Expand (96) + +- Union (95) + :- * Project (62) + : +- * Filter (61) + : +- * HashAggregate (60) + : +- Exchange (59) + : +- * HashAggregate (58) + : +- * Project (57) + : +- * BroadcastHashJoin Inner BuildRight (56) + : :- * Project (54) + : : +- * BroadcastHashJoin Inner BuildRight (53) : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) @@ -25,85 +25,80 @@ TakeOrderedAndProject (105) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - :- * Project (83) - : +- * Filter (82) - : +- * HashAggregate (81) - : +- Exchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * BroadcastHashJoin LeftSemi BuildRight (72) - : : : :- * Filter (70) - : : : : +- * ColumnarToRow (69) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (71) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - +- * Project (99) - +- * Filter (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * BroadcastHashJoin LeftSemi BuildRight (88) - : : :- * Filter (86) - : : : +- * ColumnarToRow (85) - : : : +- Scan parquet spark_catalog.default.web_sales (84) - : : +- ReusedExchange (87) - : +- ReusedExchange (89) - +- ReusedExchange (92) + : : : +- * HashAggregate (46) + : : : +- Exchange (45) + : : : +- * HashAggregate (44) + : : : +- * Project (43) + : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : :- * Project (40) + : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : :- * Filter (9) + : : : : : +- * ColumnarToRow (8) + : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : +- BroadcastExchange (38) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) + : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- BroadcastExchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (41) + : : +- ReusedExchange (52) + : +- ReusedExchange (55) + :- * Project (78) + : +- * Filter (77) + : +- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * BroadcastHashJoin LeftSemi BuildRight (67) + : : : :- * Filter (65) + : : : : +- * ColumnarToRow (64) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (63) + : : : +- ReusedExchange (66) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + +- * Project (94) + +- * Filter (93) + +- * HashAggregate (92) + +- Exchange (91) + +- * HashAggregate (90) + +- * Project (89) + +- * BroadcastHashJoin Inner BuildRight (88) + :- * Project (86) + : +- * BroadcastHashJoin Inner BuildRight (85) + : :- * BroadcastHashJoin LeftSemi BuildRight (83) + : : :- * Filter (81) + : : : +- * ColumnarToRow (80) + : : : +- Scan parquet spark_catalog.default.web_sales (79) + : : +- ReusedExchange (82) + : +- ReusedExchange (84) + +- ReusedExchange (87) (1) Scan parquet spark_catalog.default.store_sales @@ -114,10 +109,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -143,10 +138,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -157,10 +152,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -207,7 +202,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(22) ReusedExchange [Reuses operator id: 134] +(22) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 3] @@ -224,209 +219,185 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 4] +(26) BroadcastHashJoin [codegen id : 7] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(27) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(28) ColumnarToRow [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(29) Filter [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#24) -(30) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [ws_item_sk#24] +Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] +Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] +Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] -(34) Exchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(33) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#30] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] +(34) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#25] +Right keys [1]: [d_date_sk#30] +Join type: Inner +Join condition: None -(36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(35) Project [codegen id : 6] +Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] -(37) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +(36) BroadcastExchange +Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] -(38) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) +(37) BroadcastHashJoin [codegen id : 7] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] +Join type: LeftSemi +Join condition: None -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(38) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] +(39) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(40) Project [codegen id : 9] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(42) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#34] +(41) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#31] -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] +(42) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] +(43) Project [codegen id : 9] +Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] -(45) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +(44) HashAggregate [codegen id : 9] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None +(45) Exchange +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(46) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] (47) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] +Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] (50) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(52) Scan parquet spark_catalog.default.item +(52) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct -(53) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(54) Filter [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#35] - -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(57) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(58) BroadcastHashJoin [codegen id : 25] +(53) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(54) Project [codegen id : 14] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(60) ReusedExchange [Reuses operator id: 129] +(55) ReusedExchange [Reuses operator id: 124] Output [1]: [d_date_sk#40] -(61) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(57) Project [codegen id : 14] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] -(63) HashAggregate [codegen id : 25] +(58) HashAggregate [codegen id : 14] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -(64) Exchange +(59) Exchange Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(65) HashAggregate [codegen id : 26] +(60) HashAggregate [codegen id : 15] Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50] -(66) Filter [codegen id : 26] +(61) Filter [codegen id : 15] Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(67) Project [codegen id : 26] +(62) Project [codegen id : 15] Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] -(68) Scan parquet spark_catalog.default.catalog_sales +(63) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] @@ -434,75 +405,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 51] +(64) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(70) Filter [codegen id : 51] +(65) Filter [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Condition : isnotnull(cs_item_sk#57) -(71) ReusedExchange [Reuses operator id: 50] +(66) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#35] -(72) BroadcastHashJoin [codegen id : 51] +(67) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(73) ReusedExchange [Reuses operator id: 57] +(68) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(74) BroadcastHashJoin [codegen id : 51] +(69) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None -(75) Project [codegen id : 51] +(70) Project [codegen id : 29] Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#62, i_class_id#63, i_category_id#64] Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(76) ReusedExchange [Reuses operator id: 129] +(71) ReusedExchange [Reuses operator id: 124] Output [1]: [d_date_sk#65] -(77) BroadcastHashJoin [codegen id : 51] +(72) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(78) Project [codegen id : 51] +(73) Project [codegen id : 29] Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#62, i_class_id#63, i_category_id#64] Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -(79) HashAggregate [codegen id : 51] +(74) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#62, i_class_id#63, i_category_id#64] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(80) Exchange +(75) Exchange Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(81) HashAggregate [codegen id : 52] +(76) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73] Results [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75] -(82) Filter [codegen id : 52] +(77) Filter [codegen id : 30] Input [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(83) Project [codegen id : 52] +(78) Project [codegen id : 30] Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#62, i_class_id#63, i_category_id#64] Input [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sales#74, number_sales#75] -(84) Scan parquet spark_catalog.default.web_sales +(79) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] @@ -510,291 +481,291 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 77] +(80) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -(86) Filter [codegen id : 77] +(81) Filter [codegen id : 44] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Condition : isnotnull(ws_item_sk#77) -(87) ReusedExchange [Reuses operator id: 50] +(82) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#35] -(88) BroadcastHashJoin [codegen id : 77] +(83) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#77] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(89) ReusedExchange [Reuses operator id: 57] +(84) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] -(90) BroadcastHashJoin [codegen id : 77] +(85) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#77] Right keys [1]: [i_item_sk#81] Join type: Inner Join condition: None -(91) Project [codegen id : 77] +(86) Project [codegen id : 44] Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#82, i_class_id#83, i_category_id#84] Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] -(92) ReusedExchange [Reuses operator id: 129] +(87) ReusedExchange [Reuses operator id: 124] Output [1]: [d_date_sk#85] -(93) BroadcastHashJoin [codegen id : 77] +(88) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#80] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(94) Project [codegen id : 77] +(89) Project [codegen id : 44] Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#82, i_class_id#83, i_category_id#84] Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#82, i_class_id#83, i_category_id#84, d_date_sk#85] -(95) HashAggregate [codegen id : 77] +(90) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#82, i_class_id#83, i_category_id#84] Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88] Results [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -(96) Exchange +(91) Exchange Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -Arguments: hashpartitioning(i_brand_id#82, i_class_id#83, i_category_id#84, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(i_brand_id#82, i_class_id#83, i_category_id#84, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(97) HashAggregate [codegen id : 78] +(92) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93] Results [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95] -(98) Filter [codegen id : 78] +(93) Filter [codegen id : 45] Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#94, number_sales#95] Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(99) Project [codegen id : 78] +(94) Project [codegen id : 45] Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#82, i_class_id#83, i_category_id#84] Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#94, number_sales#95] -(100) Union +(95) Union -(101) Expand [codegen id : 79] +(96) Expand [codegen id : 46] Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -(102) HashAggregate [codegen id : 79] +(97) HashAggregate [codegen id : 46] Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -(103) Exchange +(98) Exchange Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(104) HashAggregate [codegen id : 80] +(99) HashAggregate [codegen id : 47] Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [sum(sales#49), sum(number_sales#50)] Aggregate Attributes [2]: [sum(sales#49)#108, sum(number_sales#50)#109] Results [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales#49)#108 AS sum(sales)#110, sum(number_sales#50)#109 AS sum(number_sales)#111] -(105) TakeOrderedAndProject +(100) TakeOrderedAndProject Input [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_class_id#99 ASC NULLS FIRST, i_category_id#100 ASC NULLS FIRST], [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* HashAggregate (124) -+- Exchange (123) - +- * HashAggregate (122) - +- Union (121) +Subquery:1 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* HashAggregate (119) ++- Exchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * Project (105) + : +- * BroadcastHashJoin Inner BuildRight (104) + : :- * ColumnarToRow (102) + : : +- Scan parquet spark_catalog.default.store_sales (101) + : +- ReusedExchange (103) :- * Project (110) : +- * BroadcastHashJoin Inner BuildRight (109) : :- * ColumnarToRow (107) - : : +- Scan parquet spark_catalog.default.store_sales (106) + : : +- Scan parquet spark_catalog.default.catalog_sales (106) : +- ReusedExchange (108) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- Scan parquet spark_catalog.default.catalog_sales (111) - : +- ReusedExchange (113) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * ColumnarToRow (117) - : +- Scan parquet spark_catalog.default.web_sales (116) - +- ReusedExchange (118) - - -(106) Scan parquet spark_catalog.default.store_sales + +- * Project (115) + +- * BroadcastHashJoin Inner BuildRight (114) + :- * ColumnarToRow (112) + : +- Scan parquet spark_catalog.default.web_sales (111) + +- ReusedExchange (113) + + +(101) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 2] +(102) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] -(108) ReusedExchange [Reuses operator id: 134] +(103) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#115] -(109) BroadcastHashJoin [codegen id : 2] +(104) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#114] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(110) Project [codegen id : 2] +(105) Project [codegen id : 2] Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117] Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115] -(111) Scan parquet spark_catalog.default.catalog_sales +(106) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] +(107) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] -(113) ReusedExchange [Reuses operator id: 134] +(108) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#121] -(114) BroadcastHashJoin [codegen id : 4] +(109) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#120] Right keys [1]: [d_date_sk#121] Join type: Inner Join condition: None -(115) Project [codegen id : 4] +(110) Project [codegen id : 4] Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123] Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121] -(116) Scan parquet spark_catalog.default.web_sales +(111) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 6] +(112) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] -(118) ReusedExchange [Reuses operator id: 134] +(113) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#127] -(119) BroadcastHashJoin [codegen id : 6] +(114) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#126] Right keys [1]: [d_date_sk#127] Join type: Inner Join condition: None -(120) Project [codegen id : 6] +(115) Project [codegen id : 6] Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129] Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127] -(121) Union +(116) Union -(122) HashAggregate [codegen id : 7] +(117) HashAggregate [codegen id : 7] Input [2]: [quantity#116, list_price#117] Keys: [] Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] Aggregate Attributes [2]: [sum#130, count#131] Results [2]: [sum#132, count#133] -(123) Exchange +(118) Exchange Input [2]: [sum#132, count#133] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(124) HashAggregate [codegen id : 8] +(119) HashAggregate [codegen id : 8] Input [2]: [sum#132, count#133] Keys: [] Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] Aggregate Attributes [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134] Results [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134 AS average_sales#135] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 106 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (129) -+- * Project (128) - +- * Filter (127) - +- * ColumnarToRow (126) - +- Scan parquet spark_catalog.default.date_dim (125) +BroadcastExchange (124) ++- * Project (123) + +- * Filter (122) + +- * ColumnarToRow (121) + +- Scan parquet spark_catalog.default.date_dim (120) -(125) Scan parquet spark_catalog.default.date_dim +(120) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#40, d_year#136, d_moy#137] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(126) ColumnarToRow [codegen id : 1] +(121) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#40, d_year#136, d_moy#137] -(127) Filter [codegen id : 1] +(122) Filter [codegen id : 1] Input [3]: [d_date_sk#40, d_year#136, d_moy#137] Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 = 2001)) AND (d_moy#137 = 11)) AND isnotnull(d_date_sk#40)) -(128) Project [codegen id : 1] +(123) Project [codegen id : 1] Output [1]: [d_date_sk#40] Input [3]: [d_date_sk#40, d_year#136, d_moy#137] -(129) BroadcastExchange +(124) BroadcastExchange Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (134) -+- * Project (133) - +- * Filter (132) - +- * ColumnarToRow (131) - +- Scan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (129) ++- * Project (128) + +- * Filter (127) + +- * ColumnarToRow (126) + +- Scan parquet spark_catalog.default.date_dim (125) -(130) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#138] +(125) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_year#138] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(131) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#138] +(126) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#138] -(132) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#138] -Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#24)) +(127) Filter [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#138] +Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#31)) -(133) Project [codegen id : 1] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#138] +(128) Project [codegen id : 1] +Output [1]: [d_date_sk#31] +Input [2]: [d_date_sk#31, d_year#138] -(134) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(129) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 77 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:11 Hosting operator id = 93 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 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 bdcfb39025bac..9e38c878c274c 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 @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) + WholeStageCodegen (47) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) + WholeStageCodegen (46) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (15) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 WholeStageCodegen (8) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #13 + Exchange #12 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -81,31 +81,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #5 WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - 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 #8 - WholeStageCodegen (4) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + 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 #8 + WholeStageCodegen (7) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -132,44 +132,36 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) + WholeStageCodegen (30) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (29) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -184,17 +176,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -209,6 +201,6 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter ReusedExchange [d_date_sk] #3 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 feab76d741807..f7de3cf995f55 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 @@ -1,12 +1,12 @@ == Physical Plan == -TakeOrderedAndProject (90) -+- * BroadcastHashJoin Inner BuildRight (89) - :- * Filter (72) - : +- * HashAggregate (71) - : +- Exchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) +TakeOrderedAndProject (88) ++- * BroadcastHashJoin Inner BuildRight (87) + :- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) : :- * Project (60) : : +- * BroadcastHashJoin Inner BuildRight (59) : : :- * BroadcastHashJoin LeftSemi BuildRight (57) @@ -67,28 +67,26 @@ TakeOrderedAndProject (90) : : : : +- ReusedExchange (44) : : : +- ReusedExchange (47) : : +- ReusedExchange (58) - : +- BroadcastExchange (66) - : +- * BroadcastHashJoin LeftSemi BuildRight (65) - : :- * Filter (63) - : : +- * ColumnarToRow (62) - : : +- Scan parquet spark_catalog.default.item (61) - : +- ReusedExchange (64) - +- BroadcastExchange (88) - +- * Filter (87) - +- * HashAggregate (86) - +- Exchange (85) - +- * HashAggregate (84) - +- * Project (83) - +- * BroadcastHashJoin Inner BuildRight (82) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * BroadcastHashJoin LeftSemi BuildRight (77) - : : :- * Filter (75) - : : : +- * ColumnarToRow (74) - : : : +- Scan parquet spark_catalog.default.store_sales (73) - : : +- ReusedExchange (76) - : +- ReusedExchange (78) - +- ReusedExchange (81) + : +- BroadcastExchange (64) + : +- * Filter (63) + : +- * ColumnarToRow (62) + : +- Scan parquet spark_catalog.default.item (61) + +- BroadcastExchange (86) + +- * Filter (85) + +- * HashAggregate (84) + +- Exchange (83) + +- * HashAggregate (82) + +- * Project (81) + +- * BroadcastHashJoin Inner BuildRight (80) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin LeftSemi BuildRight (75) + : : :- * Filter (73) + : : : +- * ColumnarToRow (72) + : : : +- Scan parquet spark_catalog.default.store_sales (71) + : : +- ReusedExchange (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -99,10 +97,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 37] +(2) ColumnarToRow [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 37] +(3) Filter [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -135,7 +133,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 123] +(10) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -185,7 +183,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 123] +(21) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -295,7 +293,7 @@ Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 123] +(44) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#30] (45) BroadcastHashJoin [codegen id : 14] @@ -353,22 +351,22 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 37] +(57) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 114] +(58) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 37] +(59) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 37] +(60) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] @@ -379,59 +377,50 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 36] +(62) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(63) Filter [codegen id : 36] +(63) Filter [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40)) -(64) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#35] - -(65) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [i_item_sk#37] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(66) BroadcastExchange +(64) BroadcastExchange Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(67) BroadcastHashJoin [codegen id : 37] +(65) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(68) Project [codegen id : 37] +(66) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(69) HashAggregate [codegen id : 37] +(67) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(70) Exchange +(68) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(71) HashAggregate [codegen id : 76] +(69) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(72) Filter [codegen id : 76] +(70) Filter [codegen id : 42] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(73) Scan parquet spark_catalog.default.store_sales +(71) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -439,285 +428,285 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(74) ColumnarToRow [codegen id : 74] +(72) ColumnarToRow [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(75) Filter [codegen id : 74] +(73) Filter [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) -(76) ReusedExchange [Reuses operator id: 56] +(74) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#35] -(77) BroadcastHashJoin [codegen id : 74] +(75) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(78) ReusedExchange [Reuses operator id: 128] +(76) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#59] -(79) BroadcastHashJoin [codegen id : 74] +(77) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(80) Project [codegen id : 74] +(78) Project [codegen id : 40] Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56] Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59] -(81) ReusedExchange [Reuses operator id: 66] +(79) ReusedExchange [Reuses operator id: 64] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(82) BroadcastHashJoin [codegen id : 74] +(80) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(83) Project [codegen id : 74] +(81) Project [codegen id : 40] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(84) HashAggregate [codegen id : 74] +(82) HashAggregate [codegen id : 40] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] -(85) Exchange +(83) Exchange Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(86) HashAggregate [codegen id : 75] +(84) HashAggregate [codegen id : 41] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(87) Filter [codegen id : 75] +(85) Filter [codegen id : 41] Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(88) BroadcastExchange +(86) BroadcastExchange Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13] -(89) BroadcastHashJoin [codegen id : 76] +(87) BroadcastHashJoin [codegen id : 42] Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner Join condition: None -(90) TakeOrderedAndProject +(88) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: 100, [i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (109) -+- Exchange (108) - +- * HashAggregate (107) - +- Union (106) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * ColumnarToRow (92) - : : +- Scan parquet spark_catalog.default.store_sales (91) - : +- ReusedExchange (93) - :- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * ColumnarToRow (97) - : : +- Scan parquet spark_catalog.default.catalog_sales (96) - : +- ReusedExchange (98) - +- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * ColumnarToRow (102) - : +- Scan parquet spark_catalog.default.web_sales (101) - +- ReusedExchange (103) - - -(91) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (107) ++- Exchange (106) + +- * HashAggregate (105) + +- Union (104) + :- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * ColumnarToRow (90) + : : +- Scan parquet spark_catalog.default.store_sales (89) + : +- ReusedExchange (91) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * ColumnarToRow (95) + : : +- Scan parquet spark_catalog.default.catalog_sales (94) + : +- ReusedExchange (96) + +- * Project (103) + +- * BroadcastHashJoin Inner BuildRight (102) + :- * ColumnarToRow (100) + : +- Scan parquet spark_catalog.default.web_sales (99) + +- ReusedExchange (101) + + +(89) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 2] +(90) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(93) ReusedExchange [Reuses operator id: 123] +(91) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#78] -(94) BroadcastHashJoin [codegen id : 2] +(92) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(95) Project [codegen id : 2] +(93) Project [codegen id : 2] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] -(96) Scan parquet spark_catalog.default.catalog_sales +(94) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 4] +(95) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -(98) ReusedExchange [Reuses operator id: 123] +(96) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#84] -(99) BroadcastHashJoin [codegen id : 4] +(97) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(100) Project [codegen id : 4] +(98) Project [codegen id : 4] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] -(101) Scan parquet spark_catalog.default.web_sales +(99) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 6] +(100) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(103) ReusedExchange [Reuses operator id: 123] +(101) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#90] -(104) BroadcastHashJoin [codegen id : 6] +(102) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(105) Project [codegen id : 6] +(103) Project [codegen id : 6] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] -(106) Union +(104) Union -(107) HashAggregate [codegen id : 7] +(105) HashAggregate [codegen id : 7] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [2]: [sum#93, count#94] Results [2]: [sum#95, count#96] -(108) Exchange +(106) Exchange Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(109) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 8] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] -Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 94 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (114) -+- * Project (113) - +- * Filter (112) - +- * ColumnarToRow (111) - +- Scan parquet spark_catalog.default.date_dim (110) +BroadcastExchange (112) ++- * Project (111) + +- * Filter (110) + +- * ColumnarToRow (109) + +- Scan parquet spark_catalog.default.date_dim (108) -(110) Scan parquet spark_catalog.default.date_dim +(108) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#36, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 1] +(109) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] -(112) Filter [codegen id : 1] +(110) Filter [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36)) -(113) Project [codegen id : 1] +(111) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [2]: [d_date_sk#36, d_week_seq#99] -(114) BroadcastExchange +(112) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:6 Hosting operator id = 112 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* Project (118) -+- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet spark_catalog.default.date_dim (115) +Subquery:6 Hosting operator id = 110 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* Project (116) ++- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet spark_catalog.default.date_dim (113) -(115) Scan parquet spark_catalog.default.date_dim +(113) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -(117) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) -(118) Project [codegen id : 1] +(116) Project [codegen id : 1] Output [1]: [d_week_seq#102] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (123) -+- * Project (122) - +- * Filter (121) - +- * ColumnarToRow (120) - +- Scan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (121) ++- * Project (120) + +- * Filter (119) + +- * ColumnarToRow (118) + +- Scan parquet spark_catalog.default.date_dim (117) -(119) Scan parquet spark_catalog.default.date_dim +(117) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] +(118) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] -(121) Filter [codegen id : 1] +(119) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#13)) -(122) Project [codegen id : 1] +(120) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#106] -(123) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] @@ -725,60 +714,60 @@ Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN d Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 73 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (128) -+- * Project (127) - +- * Filter (126) - +- * ColumnarToRow (125) - +- Scan parquet spark_catalog.default.date_dim (124) +Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (126) ++- * Project (125) + +- * Filter (124) + +- * ColumnarToRow (123) + +- Scan parquet spark_catalog.default.date_dim (122) -(124) Scan parquet spark_catalog.default.date_dim +(122) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#59, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(125) ColumnarToRow [codegen id : 1] +(123) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] -(126) Filter [codegen id : 1] +(124) Filter [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59)) -(127) Project [codegen id : 1] +(125) Project [codegen id : 1] Output [1]: [d_date_sk#59] Input [2]: [d_date_sk#59, d_week_seq#107] -(128) BroadcastExchange +(126) BroadcastExchange Input [1]: [d_date_sk#59] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -Subquery:12 Hosting operator id = 126 Hosting Expression = Subquery scalar-subquery#108, [id=#109] -* Project (132) -+- * Filter (131) - +- * ColumnarToRow (130) - +- Scan parquet spark_catalog.default.date_dim (129) +Subquery:12 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* Project (130) ++- * Filter (129) + +- * ColumnarToRow (128) + +- Scan parquet spark_catalog.default.date_dim (127) -(129) Scan parquet spark_catalog.default.date_dim +(127) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(130) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -(131) Filter [codegen id : 1] +(129) Filter [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11)) -(132) Project [codegen id : 1] +(130) Project [codegen id : 1] Output [1]: [d_week_seq#110] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] 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 0d6f76d6e73bb..005d0cc8cfb42 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 @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (76) + WholeStageCodegen (42) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 @@ -41,7 +41,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (37) + WholeStageCodegen (20) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -170,23 +170,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (36) - BroadcastHashJoin [i_item_sk,ss_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] - InputAdapter - ReusedExchange [ss_item_sk] #3 + WholeStageCodegen (19) + 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] InputAdapter BroadcastExchange #15 - WholeStageCodegen (75) + WholeStageCodegen (41) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (74) + WholeStageCodegen (40) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] 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 6d5c59fbbb128..d95e18602d24d 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 @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) +TakeOrderedAndProject (82) ++- * BroadcastHashJoin Inner BuildRight (81) + :- * Filter (64) + : +- * HashAggregate (63) + : +- Exchange (62) + : +- * HashAggregate (61) + : +- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (57) + : : +- * BroadcastHashJoin Inner BuildRight (56) : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) @@ -20,69 +20,67 @@ TakeOrderedAndProject (84) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * Filter (69) - : : : +- * ColumnarToRow (68) - : : : +- Scan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) + : : : +- * HashAggregate (46) + : : : +- Exchange (45) + : : : +- * HashAggregate (44) + : : : +- * Project (43) + : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : :- * Project (40) + : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : :- * Filter (9) + : : : : : +- * ColumnarToRow (8) + : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : +- BroadcastExchange (38) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) + : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- BroadcastExchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (41) + : : +- BroadcastExchange (55) + : : +- * Filter (54) + : : +- * ColumnarToRow (53) + : : +- Scan parquet spark_catalog.default.item (52) + : +- ReusedExchange (58) + +- BroadcastExchange (80) + +- * Filter (79) + +- * HashAggregate (78) + +- Exchange (77) + +- * HashAggregate (76) + +- * Project (75) + +- * BroadcastHashJoin Inner BuildRight (74) + :- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * BroadcastHashJoin LeftSemi BuildRight (69) + : : :- * Filter (67) + : : : +- * ColumnarToRow (66) + : : : +- Scan parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (68) + : +- ReusedExchange (70) + +- ReusedExchange (73) (1) Scan parquet spark_catalog.default.store_sales @@ -93,10 +91,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -122,10 +120,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -136,10 +134,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -186,7 +184,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(22) ReusedExchange [Reuses operator id: 117] +(22) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 3] @@ -203,127 +201,127 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 4] +(26) BroadcastHashJoin [codegen id : 7] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(27) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(28) ColumnarToRow [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(29) Filter [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#24) -(30) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [ws_item_sk#24] +Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] +Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] +Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) Exchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(33) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#30] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] +(34) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#25] +Right keys [1]: [d_date_sk#30] +Join type: Inner +Join condition: None -(36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(35) Project [codegen id : 6] +Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] -(37) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +(36) BroadcastExchange +Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] -(38) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) +(37) BroadcastHashJoin [codegen id : 7] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] +Join type: LeftSemi +Join condition: None -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(38) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] +(39) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(40) Project [codegen id : 9] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(42) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#34] +(41) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#31] -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] +(42) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] +(43) Project [codegen id : 9] +Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] + +(44) HashAggregate [codegen id : 9] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] -(45) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +(45) Exchange +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None +(46) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] (47) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] +Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] (50) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -336,72 +334,63 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 23] +(53) ColumnarToRow [codegen id : 12] Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(54) Filter [codegen id : 23] +(54) Filter [codegen id : 12] Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) -(55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#35] - -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(57) BroadcastExchange +(55) BroadcastExchange Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(57) Project [codegen id : 14] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(60) ReusedExchange [Reuses operator id: 108] +(58) ReusedExchange [Reuses operator id: 106] Output [1]: [d_date_sk#40] -(61) BroadcastHashJoin [codegen id : 25] +(59) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(60) Project [codegen id : 14] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] -(63) HashAggregate [codegen id : 25] +(61) HashAggregate [codegen id : 14] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -(64) Exchange +(62) Exchange Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(65) HashAggregate [codegen id : 52] +(63) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(66) Filter [codegen id : 52] +(64) Filter [codegen id : 30] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.store_sales +(65) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -409,346 +398,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 50] +(66) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(69) Filter [codegen id : 50] +(67) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) -(70) ReusedExchange [Reuses operator id: 50] +(68) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#35] -(71) BroadcastHashJoin [codegen id : 50] +(69) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(72) ReusedExchange [Reuses operator id: 57] +(70) ReusedExchange [Reuses operator id: 55] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(73) BroadcastHashJoin [codegen id : 50] +(71) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(74) Project [codegen id : 50] +(72) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) ReusedExchange [Reuses operator id: 122] +(73) ReusedExchange [Reuses operator id: 120] Output [1]: [d_date_sk#63] -(76) BroadcastHashJoin [codegen id : 50] +(74) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(77) Project [codegen id : 50] +(75) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -(78) HashAggregate [codegen id : 50] +(76) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(79) Exchange +(77) Exchange Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(80) HashAggregate [codegen id : 51] +(78) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(81) Filter [codegen id : 51] +(79) Filter [codegen id : 29] Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(82) BroadcastExchange +(80) BroadcastExchange Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(83) BroadcastHashJoin [codegen id : 52] +(81) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join type: Inner Join condition: None -(84) TakeOrderedAndProject +(82) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- Scan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- Scan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- Scan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) - - -(85) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (101) ++- Exchange (100) + +- * HashAggregate (99) + +- Union (98) + :- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * ColumnarToRow (84) + : : +- Scan parquet spark_catalog.default.store_sales (83) + : +- ReusedExchange (85) + :- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * ColumnarToRow (89) + : : +- Scan parquet spark_catalog.default.catalog_sales (88) + : +- ReusedExchange (90) + +- * Project (97) + +- * BroadcastHashJoin Inner BuildRight (96) + :- * ColumnarToRow (94) + : +- Scan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (95) + + +(83) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] +(84) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(87) ReusedExchange [Reuses operator id: 117] +(85) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#78] -(88) BroadcastHashJoin [codegen id : 2] +(86) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(89) Project [codegen id : 2] +(87) Project [codegen id : 2] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] -(90) Scan parquet spark_catalog.default.catalog_sales +(88) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] +(89) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -(92) ReusedExchange [Reuses operator id: 117] +(90) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#84] -(93) BroadcastHashJoin [codegen id : 4] +(91) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(94) Project [codegen id : 4] +(92) Project [codegen id : 4] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] -(95) Scan parquet spark_catalog.default.web_sales +(93) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] +(94) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(97) ReusedExchange [Reuses operator id: 117] +(95) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#90] -(98) BroadcastHashJoin [codegen id : 6] +(96) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(99) Project [codegen id : 6] +(97) Project [codegen id : 6] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] -(100) Union +(98) Union -(101) HashAggregate [codegen id : 7] +(99) HashAggregate [codegen id : 7] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [2]: [sum#93, count#94] Results [2]: [sum#95, count#96] -(102) Exchange +(100) Exchange Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(103) HashAggregate [codegen id : 8] +(101) HashAggregate [codegen id : 8] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 88 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * Project (107) - +- * Filter (106) - +- * ColumnarToRow (105) - +- Scan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (106) ++- * Project (105) + +- * Filter (104) + +- * ColumnarToRow (103) + +- Scan parquet spark_catalog.default.date_dim (102) -(104) Scan parquet spark_catalog.default.date_dim +(102) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#40, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_week_seq#99] -(106) Filter [codegen id : 1] +(104) Filter [codegen id : 1] Input [2]: [d_date_sk#40, d_week_seq#99] Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#40)) -(107) Project [codegen id : 1] +(105) Project [codegen id : 1] Output [1]: [d_date_sk#40] Input [2]: [d_date_sk#40, d_week_seq#99] -(108) BroadcastExchange +(106) BroadcastExchange Input [1]: [d_date_sk#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 106 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* Project (112) -+- * Filter (111) - +- * ColumnarToRow (110) - +- Scan parquet spark_catalog.default.date_dim (109) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* Project (110) ++- * Filter (109) + +- * ColumnarToRow (108) + +- Scan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim +(107) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(110) ColumnarToRow [codegen id : 1] +(108) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -(111) Filter [codegen id : 1] +(109) Filter [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) -(112) Project [codegen id : 1] +(110) Project [codegen id : 1] Output [1]: [d_week_seq#102] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * Project (116) - +- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (115) ++- * Project (114) + +- * Filter (113) + +- * ColumnarToRow (112) + +- Scan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#106] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#106] +(112) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#106] -(115) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#106] -Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#24)) +(113) Filter [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#106] +Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#31)) -(116) Project [codegen id : 1] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#106] +(114) Project [codegen id : 1] +Output [1]: [d_date_sk#31] +Input [2]: [d_date_sk#31, d_year#106] -(117) BroadcastExchange -Input [1]: [d_date_sk#24] +(115) BroadcastExchange +Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (122) -+- * Project (121) - +- * Filter (120) - +- * ColumnarToRow (119) - +- Scan parquet spark_catalog.default.date_dim (118) +Subquery:11 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (120) ++- * Project (119) + +- * Filter (118) + +- * ColumnarToRow (117) + +- Scan parquet spark_catalog.default.date_dim (116) -(118) Scan parquet spark_catalog.default.date_dim +(116) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#63, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(119) ColumnarToRow [codegen id : 1] +(117) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#63, d_week_seq#107] -(120) Filter [codegen id : 1] +(118) Filter [codegen id : 1] Input [2]: [d_date_sk#63, d_week_seq#107] Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#63)) -(121) Project [codegen id : 1] +(119) Project [codegen id : 1] Output [1]: [d_date_sk#63] Input [2]: [d_date_sk#63, d_week_seq#107] -(122) BroadcastExchange +(120) BroadcastExchange Input [1]: [d_date_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 120 Hosting Expression = Subquery scalar-subquery#108, [id=#109] -* Project (126) -+- * Filter (125) - +- * ColumnarToRow (124) - +- Scan parquet spark_catalog.default.date_dim (123) +Subquery:12 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* Project (124) ++- * Filter (123) + +- * ColumnarToRow (122) + +- Scan parquet spark_catalog.default.date_dim (121) -(123) Scan parquet spark_catalog.default.date_dim +(121) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(124) ColumnarToRow [codegen id : 1] +(122) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -(125) Filter [codegen id : 1] +(123) Filter [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11)) -(126) Project [codegen id : 1] +(124) Project [codegen id : 1] Output [1]: [d_week_seq#110] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] 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 fb2e53c91965a..825bb2d20a585 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 @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 @@ -41,7 +41,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -79,31 +79,31 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - 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 #7 - WholeStageCodegen (4) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + 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 #7 + WholeStageCodegen (7) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -130,45 +130,42 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_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] - InputAdapter - ReusedExchange [ss_item_sk] #3 + WholeStageCodegen (12) + 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] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 4249b82d0da97..3c0b2b79f58de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -52,7 +52,7 @@ Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -60,7 +60,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : (((((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(cs_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(cs_call_center_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(cs_ship_date_sk#1, 42))) +Condition : (((((((isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) AND isnotnull(cs_ship_date_sk#1)) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(cs_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(cs_call_center_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(cs_ship_date_sk#1, 42))) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 2c6a10e532f17..c87ec0e52c418 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] + Filter [cs_order_number,cs_warehouse_sk,cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 17961, 333176, 0, 0),bloomFilter,buf] Exchange #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 796995db700ff..ed251aca1c521 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -52,7 +52,7 @@ Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] +PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -60,7 +60,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) +Condition : ((((isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) AND isnotnull(cs_ship_date_sk#1)) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index a2022d0bff800..b639fb0aed326 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] + Filter [cs_order_number,cs_warehouse_sk,cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index f5a2d52b1d900..1efaa573ec1f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -TakeOrderedAndProject (129) -+- Union (128) - :- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * SortMergeJoin Inner (69) +TakeOrderedAndProject (105) ++- Union (104) + :- * HashAggregate (61) + : +- Exchange (60) + : +- * HashAggregate (59) + : +- * Project (58) + : +- * SortMergeJoin Inner (57) : :- * Project (51) : : +- * BroadcastHashJoin Inner BuildRight (50) : : :- * SortMergeJoin LeftSemi (48) @@ -57,77 +57,53 @@ TakeOrderedAndProject (129) : : : +- * ColumnarToRow (37) : : : +- Scan parquet spark_catalog.default.customer (36) : : +- ReusedExchange (49) - : +- * SortMergeJoin LeftSemi (68) - : :- * Sort (56) - : : +- Exchange (55) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet spark_catalog.default.customer (52) - : +- * Sort (67) - : +- * Project (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * SortMergeJoin Inner (61) - : :- * Sort (58) - : : +- ReusedExchange (57) - : +- * Sort (60) - : +- ReusedExchange (59) - +- * HashAggregate (127) - +- Exchange (126) - +- * HashAggregate (125) - +- * Project (124) - +- * SortMergeJoin Inner (123) - :- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * SortMergeJoin LeftSemi (105) - : : :- * Sort (93) - : : : +- Exchange (92) - : : : +- * Project (91) - : : : +- * SortMergeJoin LeftSemi (90) - : : : :- * Sort (78) - : : : : +- Exchange (77) - : : : : +- * Filter (76) - : : : : +- * ColumnarToRow (75) - : : : : +- Scan parquet spark_catalog.default.web_sales (74) - : : : +- * Sort (89) - : : : +- * Project (88) - : : : +- * Filter (87) - : : : +- * HashAggregate (86) - : : : +- * HashAggregate (85) - : : : +- * Project (84) - : : : +- * SortMergeJoin Inner (83) - : : : :- * Sort (80) - : : : : +- ReusedExchange (79) - : : : +- * Sort (82) - : : : +- ReusedExchange (81) - : : +- * Sort (104) - : : +- * Project (103) - : : +- * Filter (102) - : : +- * HashAggregate (101) - : : +- * HashAggregate (100) - : : +- * Project (99) - : : +- * SortMergeJoin Inner (98) - : : :- * Sort (95) - : : : +- ReusedExchange (94) - : : +- * Sort (97) - : : +- ReusedExchange (96) - : +- ReusedExchange (106) - +- * SortMergeJoin LeftSemi (122) - :- * Sort (110) - : +- ReusedExchange (109) - +- * Sort (121) - +- * Project (120) - +- * Filter (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- * Project (116) - +- * SortMergeJoin Inner (115) - :- * Sort (112) - : +- ReusedExchange (111) - +- * Sort (114) - +- ReusedExchange (113) + : +- * Sort (56) + : +- Exchange (55) + : +- * Filter (54) + : +- * ColumnarToRow (53) + : +- Scan parquet spark_catalog.default.customer (52) + +- * HashAggregate (103) + +- Exchange (102) + +- * HashAggregate (101) + +- * Project (100) + +- * SortMergeJoin Inner (99) + :- * Project (96) + : +- * BroadcastHashJoin Inner BuildRight (95) + : :- * SortMergeJoin LeftSemi (93) + : : :- * Sort (81) + : : : +- Exchange (80) + : : : +- * Project (79) + : : : +- * SortMergeJoin LeftSemi (78) + : : : :- * Sort (66) + : : : : +- Exchange (65) + : : : : +- * Filter (64) + : : : : +- * ColumnarToRow (63) + : : : : +- Scan parquet spark_catalog.default.web_sales (62) + : : : +- * Sort (77) + : : : +- * Project (76) + : : : +- * Filter (75) + : : : +- * HashAggregate (74) + : : : +- * HashAggregate (73) + : : : +- * Project (72) + : : : +- * SortMergeJoin Inner (71) + : : : :- * Sort (68) + : : : : +- ReusedExchange (67) + : : : +- * Sort (70) + : : : +- ReusedExchange (69) + : : +- * Sort (92) + : : +- * Project (91) + : : +- * Filter (90) + : : +- * HashAggregate (89) + : : +- * HashAggregate (88) + : : +- * Project (87) + : : +- * SortMergeJoin Inner (86) + : : :- * Sort (83) + : : : +- ReusedExchange (82) + : : +- * Sort (85) + : : +- ReusedExchange (84) + : +- ReusedExchange (94) + +- * Sort (98) + +- ReusedExchange (97) (1) Scan parquet spark_catalog.default.catalog_sales @@ -135,7 +111,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -143,7 +119,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) (4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -168,7 +144,7 @@ Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(9) ReusedExchange [Reuses operator id: 139] +(9) ReusedExchange [Reuses operator id: 115] Output [2]: [d_date_sk#10, d_date#11] (10) BroadcastHashJoin [codegen id : 4] @@ -355,7 +331,7 @@ Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(49) ReusedExchange [Reuses operator id: 134] +(49) ReusedExchange [Reuses operator id: 110] Output [1]: [d_date_sk#33] (50) BroadcastHashJoin [codegen id : 17] @@ -390,430 +366,318 @@ Arguments: hashpartitioning(c_customer_sk#34, 5), ENSURE_REQUIREMENTS, [plan_id= Input [3]: [c_customer_sk#34, c_first_name#35, c_last_name#36] Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 -(57) ReusedExchange [Reuses operator id: 34] -Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(58) Sort [codegen id : 21] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Arguments: [ss_customer_sk#20 ASC NULLS FIRST], false, 0 - -(59) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#24] - -(60) Sort [codegen id : 23] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(61) SortMergeJoin [codegen id : 24] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 24] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(63) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(64) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] - -(65) Filter [codegen id : 24] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) - -(66) Project [codegen id : 24] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(67) Sort [codegen id : 24] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_customer_sk#34] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None - -(69) SortMergeJoin [codegen id : 26] +(57) SortMergeJoin [codegen id : 20] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(70) Project [codegen id : 26] +(58) Project [codegen id : 20] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#35, c_last_name#36] Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#34, c_first_name#35, c_last_name#36] -(71) HashAggregate [codegen id : 26] +(59) HashAggregate [codegen id : 20] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#35, c_last_name#36] Keys [2]: [c_last_name#36, c_first_name#35] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [2]: [sum#37, isEmpty#38] Results [4]: [c_last_name#36, c_first_name#35, sum#39, isEmpty#40] -(72) Exchange +(60) Exchange Input [4]: [c_last_name#36, c_first_name#35, sum#39, isEmpty#40] Arguments: hashpartitioning(c_last_name#36, c_first_name#35, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(73) HashAggregate [codegen id : 27] +(61) HashAggregate [codegen id : 21] Input [4]: [c_last_name#36, c_first_name#35, sum#39, isEmpty#40] Keys [2]: [c_last_name#36, c_first_name#35] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] Results [3]: [c_last_name#36, c_first_name#35, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] -(74) Scan parquet spark_catalog.default.web_sales +(62) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 28] +(63) ColumnarToRow [codegen id : 22] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(76) Filter [codegen id : 28] +(64) Filter [codegen id : 22] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : (isnotnull(ws_bill_customer_sk#44) AND isnotnull(ws_item_sk#43)) +Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(77) Exchange +(65) Exchange Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(78) Sort [codegen id : 29] +(66) Sort [codegen id : 23] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 -(79) ReusedExchange [Reuses operator id: 12] +(67) ReusedExchange [Reuses operator id: 12] Output [2]: [ss_item_sk#7, d_date#11] -(80) Sort [codegen id : 32] +(68) Sort [codegen id : 26] Input [2]: [ss_item_sk#7, d_date#11] Arguments: [ss_item_sk#7 ASC NULLS FIRST], false, 0 -(81) ReusedExchange [Reuses operator id: 17] +(69) ReusedExchange [Reuses operator id: 17] Output [2]: [i_item_sk#12, i_item_desc#13] -(82) Sort [codegen id : 34] +(70) Sort [codegen id : 28] Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: [i_item_sk#12 ASC NULLS FIRST], false, 0 -(83) SortMergeJoin [codegen id : 35] +(71) SortMergeJoin [codegen id : 29] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(84) Project [codegen id : 35] +(72) Project [codegen id : 29] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(85) HashAggregate [codegen id : 35] +(73) HashAggregate [codegen id : 29] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#15] Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(86) HashAggregate [codegen id : 35] +(74) HashAggregate [codegen id : 29] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(87) Filter [codegen id : 35] +(75) Filter [codegen id : 29] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(88) Project [codegen id : 35] +(76) Project [codegen id : 29] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(89) Sort [codegen id : 35] +(77) Sort [codegen id : 29] Input [1]: [item_sk#18] Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 36] +(78) SortMergeJoin [codegen id : 30] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(91) Project [codegen id : 36] +(79) Project [codegen id : 30] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(92) Exchange +(80) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(93) Sort [codegen id : 37] +(81) Sort [codegen id : 31] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(94) ReusedExchange [Reuses operator id: 34] +(82) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(95) Sort [codegen id : 39] +(83) Sort [codegen id : 33] Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] Arguments: [ss_customer_sk#20 ASC NULLS FIRST], false, 0 -(96) ReusedExchange [Reuses operator id: 39] +(84) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#24] -(97) Sort [codegen id : 41] +(85) Sort [codegen id : 35] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin [codegen id : 42] +(86) SortMergeJoin [codegen id : 36] Left keys [1]: [ss_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(99) Project [codegen id : 42] +(87) Project [codegen id : 36] Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(100) HashAggregate [codegen id : 42] +(88) HashAggregate [codegen id : 36] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(101) HashAggregate [codegen id : 42] +(89) HashAggregate [codegen id : 36] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(102) Filter [codegen id : 42] +(90) Filter [codegen id : 36] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(103) Project [codegen id : 42] +(91) Project [codegen id : 36] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(104) Sort [codegen id : 42] +(92) Sort [codegen id : 36] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(105) SortMergeJoin [codegen id : 44] +(93) SortMergeJoin [codegen id : 38] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(106) ReusedExchange [Reuses operator id: 134] +(94) ReusedExchange [Reuses operator id: 110] Output [1]: [d_date_sk#48] -(107) BroadcastHashJoin [codegen id : 44] +(95) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(108) Project [codegen id : 44] +(96) Project [codegen id : 38] Output [3]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] Input [5]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#48] -(109) ReusedExchange [Reuses operator id: 55] +(97) ReusedExchange [Reuses operator id: 55] Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] -(110) Sort [codegen id : 46] +(98) Sort [codegen id : 40] Input [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] Arguments: [c_customer_sk#49 ASC NULLS FIRST], false, 0 -(111) ReusedExchange [Reuses operator id: 34] -Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(112) Sort [codegen id : 48] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Arguments: [ss_customer_sk#20 ASC NULLS FIRST], false, 0 - -(113) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#24] - -(114) Sort [codegen id : 50] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(115) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None - -(116) Project [codegen id : 51] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(117) HashAggregate [codegen id : 51] -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(118) HashAggregate [codegen id : 51] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] - -(119) Filter [codegen id : 51] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) - -(120) Project [codegen id : 51] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(121) Sort [codegen id : 51] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(122) SortMergeJoin [codegen id : 52] -Left keys [1]: [c_customer_sk#49] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None - -(123) SortMergeJoin [codegen id : 53] +(99) SortMergeJoin [codegen id : 41] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#49] Join type: Inner Join condition: None -(124) Project [codegen id : 53] +(100) Project [codegen id : 41] Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] Input [6]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, c_customer_sk#49, c_first_name#50, c_last_name#51] -(125) HashAggregate [codegen id : 53] +(101) HashAggregate [codegen id : 41] Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] Keys [2]: [c_last_name#51, c_first_name#50] Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [2]: [sum#52, isEmpty#53] Results [4]: [c_last_name#51, c_first_name#50, sum#54, isEmpty#55] -(126) Exchange +(102) Exchange Input [4]: [c_last_name#51, c_first_name#50, sum#54, isEmpty#55] Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(127) HashAggregate [codegen id : 54] +(103) HashAggregate [codegen id : 42] Input [4]: [c_last_name#51, c_first_name#50, sum#54, isEmpty#55] Keys [2]: [c_last_name#51, c_first_name#50] Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56] Results [3]: [c_last_name#51, c_first_name#50, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56 AS sales#57] -(128) Union +(104) Union -(129) TakeOrderedAndProject +(105) TakeOrderedAndProject Input [3]: [c_last_name#36, c_first_name#35, sales#42] Arguments: 100, [c_last_name#36 ASC NULLS FIRST, c_first_name#35 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#36, c_first_name#35, sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (134) -+- * Project (133) - +- * Filter (132) - +- * ColumnarToRow (131) - +- Scan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (110) ++- * Project (109) + +- * Filter (108) + +- * ColumnarToRow (107) + +- Scan parquet spark_catalog.default.date_dim (106) -(130) Scan parquet spark_catalog.default.date_dim +(106) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#33, d_year#58, d_moy#59] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(131) ColumnarToRow [codegen id : 1] +(107) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#33, d_year#58, d_moy#59] -(132) Filter [codegen id : 1] +(108) Filter [codegen id : 1] Input [3]: [d_date_sk#33, d_year#58, d_moy#59] Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#33)) -(133) Project [codegen id : 1] +(109) Project [codegen id : 1] Output [1]: [d_date_sk#33] Input [3]: [d_date_sk#33, d_year#58, d_moy#59] -(134) BroadcastExchange +(110) BroadcastExchange Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (139) -+- * Project (138) - +- * Filter (137) - +- * ColumnarToRow (136) - +- Scan parquet spark_catalog.default.date_dim (135) +BroadcastExchange (115) ++- * Project (114) + +- * Filter (113) + +- * ColumnarToRow (112) + +- Scan parquet spark_catalog.default.date_dim (111) -(135) Scan parquet spark_catalog.default.date_dim +(111) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(136) ColumnarToRow [codegen id : 1] +(112) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#60] -(137) Filter [codegen id : 1] +(113) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#60] Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(138) Project [codegen id : 1] +(114) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#60] -(139) BroadcastExchange +(115) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (156) -+- Exchange (155) - +- * HashAggregate (154) - +- * HashAggregate (153) - +- * HashAggregate (152) - +- * Project (151) - +- * SortMergeJoin Inner (150) - :- * Sort (147) - : +- Exchange (146) - : +- * Project (145) - : +- * BroadcastHashJoin Inner BuildRight (144) - : :- * Filter (142) - : : +- * ColumnarToRow (141) - : : +- Scan parquet spark_catalog.default.store_sales (140) - : +- ReusedExchange (143) - +- * Sort (149) - +- ReusedExchange (148) - - -(140) Scan parquet spark_catalog.default.store_sales +* HashAggregate (132) ++- Exchange (131) + +- * HashAggregate (130) + +- * HashAggregate (129) + +- * HashAggregate (128) + +- * Project (127) + +- * SortMergeJoin Inner (126) + :- * Sort (123) + : +- Exchange (122) + : +- * Project (121) + : +- * BroadcastHashJoin Inner BuildRight (120) + : :- * Filter (118) + : : +- * ColumnarToRow (117) + : : +- Scan parquet spark_catalog.default.store_sales (116) + : +- ReusedExchange (119) + +- * Sort (125) + +- ReusedExchange (124) + + +(116) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] @@ -821,119 +685,115 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(141) ColumnarToRow [codegen id : 2] +(117) ColumnarToRow [codegen id : 2] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -(142) Filter [codegen id : 2] +(118) Filter [codegen id : 2] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Condition : isnotnull(ss_customer_sk#61) -(143) ReusedExchange [Reuses operator id: 161] +(119) ReusedExchange [Reuses operator id: 137] Output [1]: [d_date_sk#66] -(144) BroadcastHashJoin [codegen id : 2] +(120) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#64] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(145) Project [codegen id : 2] +(121) Project [codegen id : 2] Output [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63] Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, d_date_sk#66] -(146) Exchange +(122) Exchange Input [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63] Arguments: hashpartitioning(ss_customer_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(147) Sort [codegen id : 3] +(123) Sort [codegen id : 3] Input [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63] Arguments: [ss_customer_sk#61 ASC NULLS FIRST], false, 0 -(148) ReusedExchange [Reuses operator id: 39] +(124) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#67] -(149) Sort [codegen id : 5] +(125) Sort [codegen id : 5] Input [1]: [c_customer_sk#67] Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 -(150) SortMergeJoin [codegen id : 6] +(126) SortMergeJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#61] Right keys [1]: [c_customer_sk#67] Join type: Inner Join condition: None -(151) Project [codegen id : 6] +(127) Project [codegen id : 6] Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#67] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, c_customer_sk#67] -(152) HashAggregate [codegen id : 6] +(128) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#67] Keys [1]: [c_customer_sk#67] Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] Aggregate Attributes [2]: [sum#68, isEmpty#69] Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] -(153) HashAggregate [codegen id : 6] +(129) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] Keys [1]: [c_customer_sk#67] Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72] Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73] -(154) HashAggregate [codegen id : 6] +(130) HashAggregate [codegen id : 6] Input [1]: [csales#73] Keys: [] Functions [1]: [partial_max(csales#73)] Aggregate Attributes [1]: [max#74] Results [1]: [max#75] -(155) Exchange +(131) Exchange Input [1]: [max#75] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(156) HashAggregate [codegen id : 7] +(132) HashAggregate [codegen id : 7] Input [1]: [max#75] Keys: [] Functions [1]: [max(csales#73)] Aggregate Attributes [1]: [max(csales#73)#76] Results [1]: [max(csales#73)#76 AS tpcds_cmax#77] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 -BroadcastExchange (161) -+- * Project (160) - +- * Filter (159) - +- * ColumnarToRow (158) - +- Scan parquet spark_catalog.default.date_dim (157) +Subquery:4 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 +BroadcastExchange (137) ++- * Project (136) + +- * Filter (135) + +- * ColumnarToRow (134) + +- Scan parquet spark_catalog.default.date_dim (133) -(157) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#66, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(158) ColumnarToRow [codegen id : 1] +(134) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#66, d_year#78] -(159) Filter [codegen id : 1] +(135) Filter [codegen id : 1] Input [2]: [d_date_sk#66, d_year#78] Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) -(160) Project [codegen id : 1] +(136) Project [codegen id : 1] Output [1]: [d_date_sk#66] Input [2]: [d_date_sk#66, d_year#78] -(161) BroadcastExchange +(137) BroadcastExchange Input [1]: [d_date_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:5 Hosting operator id = 62 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:8 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 90 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index 2b137f14d458c..e6d5cafb988c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (27) + WholeStageCodegen (21) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (26) + WholeStageCodegen (20) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] @@ -27,7 +27,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -151,72 +151,49 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - WholeStageCodegen (25) - SortMergeJoin [c_customer_sk,c_customer_sk] + WholeStageCodegen (19) + Sort [c_customer_sk] InputAdapter - WholeStageCodegen (19) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #13 - WholeStageCodegen (18) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - WholeStageCodegen (24) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (21) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 - InputAdapter - WholeStageCodegen (23) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #9 - WholeStageCodegen (54) + Exchange [c_customer_sk] #13 + WholeStageCodegen (18) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (42) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #14 - WholeStageCodegen (53) + WholeStageCodegen (41) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (44) + WholeStageCodegen (38) Project [ws_bill_customer_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (37) + WholeStageCodegen (31) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (36) + WholeStageCodegen (30) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_item_sk,item_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (23) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #16 - WholeStageCodegen (28) - Filter [ws_bill_customer_sk,ws_item_sk] + WholeStageCodegen (22) + Filter [ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (35) + WholeStageCodegen (29) Sort [item_sk] Project [item_sk] Filter [cnt] @@ -225,17 +202,17 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (32) + WholeStageCodegen (26) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk,d_date] #5 InputAdapter - WholeStageCodegen (34) + WholeStageCodegen (28) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 InputAdapter - WholeStageCodegen (42) + WholeStageCodegen (36) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] @@ -245,42 +222,19 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (39) + WholeStageCodegen (33) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (41) + WholeStageCodegen (35) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - WholeStageCodegen (52) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (46) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 + WholeStageCodegen (40) + Sort [c_customer_sk] InputAdapter - WholeStageCodegen (51) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (48) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 - InputAdapter - WholeStageCodegen (50) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #9 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index f70d8336ca3e9..f9d6669882cc4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- Union (86) - :- * HashAggregate (62) - : +- Exchange (61) - : +- * HashAggregate (60) - : +- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (79) ++- Union (78) + :- * HashAggregate (54) + : +- Exchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (48) + : : +- * BroadcastHashJoin Inner BuildRight (47) : : :- * SortMergeJoin LeftSemi (42) : : : :- * Sort (25) : : : : +- Exchange (24) @@ -50,42 +50,34 @@ TakeOrderedAndProject (87) : : : +- * Filter (32) : : : +- * ColumnarToRow (31) : : : +- Scan parquet spark_catalog.default.customer (30) - : : +- BroadcastExchange (54) - : : +- * SortMergeJoin LeftSemi (53) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Filter (45) - : : : +- * ColumnarToRow (44) - : : : +- Scan parquet spark_catalog.default.customer (43) - : : +- * Sort (52) - : : +- * Project (51) - : : +- * Filter (50) - : : +- * HashAggregate (49) - : : +- ReusedExchange (48) - : +- ReusedExchange (57) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * SortMergeJoin LeftSemi (76) - : : :- * Sort (70) - : : : +- Exchange (69) - : : : +- * Project (68) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (67) - : : : :- * Filter (65) - : : : : +- * ColumnarToRow (64) - : : : : +- Scan parquet spark_catalog.default.web_sales (63) - : : : +- ReusedExchange (66) - : : +- * Sort (75) - : : +- * Project (74) - : : +- * Filter (73) - : : +- * HashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) + : : +- BroadcastExchange (46) + : : +- * Filter (45) + : : +- * ColumnarToRow (44) + : : +- Scan parquet spark_catalog.default.customer (43) + : +- ReusedExchange (49) + +- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- * Project (71) + : +- * BroadcastHashJoin Inner BuildRight (70) + : :- * SortMergeJoin LeftSemi (68) + : : :- * Sort (62) + : : : +- Exchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (59) + : : : :- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet spark_catalog.default.web_sales (55) + : : : +- ReusedExchange (58) + : : +- * Sort (67) + : : +- * Project (66) + : : +- * Filter (65) + : : +- * HashAggregate (64) + : : +- ReusedExchange (63) + : +- ReusedExchange (69) + +- ReusedExchange (72) (1) Scan parquet spark_catalog.default.catalog_sales @@ -93,7 +85,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] @@ -101,7 +93,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] @@ -118,7 +110,7 @@ Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(7) ReusedExchange [Reuses operator id: 97] +(7) ReusedExchange [Reuses operator id: 89] Output [2]: [d_date_sk#10, d_date#11] (8) BroadcastHashJoin [codegen id : 3] @@ -283,7 +275,7 @@ Input [2]: [c_customer_sk#24, ssales#30] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 17] +(42) SortMergeJoin [codegen id : 12] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi @@ -303,282 +295,246 @@ Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Condition : isnotnull(c_customer_sk#33) -(46) Exchange +(46) BroadcastExchange Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(47) Sort [codegen id : 11] -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#33 ASC NULLS FIRST], false, 0 - -(48) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] - -(49) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] - -(50) Filter [codegen id : 14] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) - -(51) Project [codegen id : 14] -Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] - -(52) Sort [codegen id : 14] -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 - -(53) SortMergeJoin [codegen id : 15] -Left keys [1]: [c_customer_sk#33] -Right keys [1]: [c_customer_sk#24] -Join type: LeftSemi -Join condition: None - -(54) BroadcastExchange -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 17] +(47) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(56) Project [codegen id : 17] +(48) Project [codegen id : 12] Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] -(57) ReusedExchange [Reuses operator id: 92] +(49) ReusedExchange [Reuses operator id: 84] Output [1]: [d_date_sk#36] -(58) BroadcastHashJoin [codegen id : 17] +(50) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(51) Project [codegen id : 12] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] -(60) HashAggregate [codegen id : 17] +(52) HashAggregate [codegen id : 12] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [2]: [sum#37, isEmpty#38] Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -(61) Exchange +(53) Exchange Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(62) HashAggregate [codegen id : 18] +(54) HashAggregate [codegen id : 13] Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] -(63) Scan parquet spark_catalog.default.web_sales +(55) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 23] +(56) ColumnarToRow [codegen id : 18] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(65) Filter [codegen id : 23] +(57) Filter [codegen id : 18] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : (isnotnull(ws_bill_customer_sk#44) AND isnotnull(ws_item_sk#43)) +Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(66) ReusedExchange [Reuses operator id: 21] +(58) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#18] -(67) BroadcastHashJoin [codegen id : 23] +(59) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(68) Project [codegen id : 23] +(60) Project [codegen id : 18] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(69) Exchange +(61) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(70) Sort [codegen id : 24] +(62) Sort [codegen id : 19] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(71) ReusedExchange [Reuses operator id: 37] +(63) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(72) HashAggregate [codegen id : 27] +(64) HashAggregate [codegen id : 22] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(73) Filter [codegen id : 27] +(65) Filter [codegen id : 22] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(74) Project [codegen id : 27] +(66) Project [codegen id : 22] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(75) Sort [codegen id : 27] +(67) Sort [codegen id : 22] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin [codegen id : 35] +(68) SortMergeJoin [codegen id : 25] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(77) ReusedExchange [Reuses operator id: 54] +(69) ReusedExchange [Reuses operator id: 46] Output [3]: [c_customer_sk#48, c_first_name#49, c_last_name#50] -(78) BroadcastHashJoin [codegen id : 35] +(70) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#48] Join type: Inner Join condition: None -(79) Project [codegen id : 35] +(71) Project [codegen id : 25] Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#49, c_last_name#50] Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#48, c_first_name#49, c_last_name#50] -(80) ReusedExchange [Reuses operator id: 92] +(72) ReusedExchange [Reuses operator id: 84] Output [1]: [d_date_sk#51] -(81) BroadcastHashJoin [codegen id : 35] +(73) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(82) Project [codegen id : 35] +(74) Project [codegen id : 25] Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#49, c_last_name#50] Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#49, c_last_name#50, d_date_sk#51] -(83) HashAggregate [codegen id : 35] +(75) HashAggregate [codegen id : 25] Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#49, c_last_name#50] Keys [2]: [c_last_name#50, c_first_name#49] Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [2]: [sum#52, isEmpty#53] Results [4]: [c_last_name#50, c_first_name#49, sum#54, isEmpty#55] -(84) Exchange +(76) Exchange Input [4]: [c_last_name#50, c_first_name#49, sum#54, isEmpty#55] -Arguments: hashpartitioning(c_last_name#50, c_first_name#49, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(c_last_name#50, c_first_name#49, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(85) HashAggregate [codegen id : 36] +(77) HashAggregate [codegen id : 26] Input [4]: [c_last_name#50, c_first_name#49, sum#54, isEmpty#55] Keys [2]: [c_last_name#50, c_first_name#49] Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56] Results [3]: [c_last_name#50, c_first_name#49, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56 AS sales#57] -(86) Union +(78) Union -(87) TakeOrderedAndProject +(79) TakeOrderedAndProject Input [3]: [c_last_name#35, c_first_name#34, sales#42] Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (92) -+- * Project (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (84) ++- * Project (83) + +- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet spark_catalog.default.date_dim (80) -(88) Scan parquet spark_catalog.default.date_dim +(80) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#36, d_year#58, d_moy#59] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#36, d_year#58, d_moy#59] -(90) Filter [codegen id : 1] +(82) Filter [codegen id : 1] Input [3]: [d_date_sk#36, d_year#58, d_moy#59] Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#36)) -(91) Project [codegen id : 1] +(83) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [3]: [d_date_sk#36, d_year#58, d_moy#59] -(92) BroadcastExchange +(84) BroadcastExchange Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (97) -+- * Project (96) - +- * Filter (95) - +- * ColumnarToRow (94) - +- Scan parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * Project (88) + +- * Filter (87) + +- * ColumnarToRow (86) + +- Scan parquet spark_catalog.default.date_dim (85) -(93) Scan parquet spark_catalog.default.date_dim +(85) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 1] +(86) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#60] -(95) Filter [codegen id : 1] +(87) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#60] Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 1] +(88) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#60] -(97) BroadcastExchange +(89) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (112) -+- Exchange (111) - +- * HashAggregate (110) - +- * HashAggregate (109) - +- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * BroadcastHashJoin Inner BuildRight (105) - :- * Project (103) - : +- * BroadcastHashJoin Inner BuildRight (102) - : :- * Filter (100) - : : +- * ColumnarToRow (99) - : : +- Scan parquet spark_catalog.default.store_sales (98) - : +- ReusedExchange (101) - +- ReusedExchange (104) - - -(98) Scan parquet spark_catalog.default.store_sales +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Filter (92) + : : +- * ColumnarToRow (91) + : : +- Scan parquet spark_catalog.default.store_sales (90) + : +- ReusedExchange (93) + +- ReusedExchange (96) + + +(90) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] @@ -586,109 +542,107 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 3] +(91) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -(100) Filter [codegen id : 3] +(92) Filter [codegen id : 3] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Condition : isnotnull(ss_customer_sk#61) -(101) ReusedExchange [Reuses operator id: 33] +(93) ReusedExchange [Reuses operator id: 33] Output [1]: [c_customer_sk#66] -(102) BroadcastHashJoin [codegen id : 3] +(94) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#61] Right keys [1]: [c_customer_sk#66] Join type: Inner Join condition: None -(103) Project [codegen id : 3] +(95) Project [codegen id : 3] Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] -(104) ReusedExchange [Reuses operator id: 117] +(96) ReusedExchange [Reuses operator id: 109] Output [1]: [d_date_sk#67] -(105) BroadcastHashJoin [codegen id : 3] +(97) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#64] Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(106) Project [codegen id : 3] +(98) Project [codegen id : 3] Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67] -(107) HashAggregate [codegen id : 3] +(99) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] Keys [1]: [c_customer_sk#66] Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] Aggregate Attributes [2]: [sum#68, isEmpty#69] Results [3]: [c_customer_sk#66, sum#70, isEmpty#71] -(108) Exchange +(100) Exchange Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(109) HashAggregate [codegen id : 4] +(101) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] Keys [1]: [c_customer_sk#66] Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72] Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73] -(110) HashAggregate [codegen id : 4] +(102) HashAggregate [codegen id : 4] Input [1]: [csales#73] Keys: [] Functions [1]: [partial_max(csales#73)] Aggregate Attributes [1]: [max#74] Results [1]: [max#75] -(111) Exchange +(103) Exchange Input [1]: [max#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(112) HashAggregate [codegen id : 5] +(104) HashAggregate [codegen id : 5] Input [1]: [max#75] Keys: [] Functions [1]: [max(csales#73)] Aggregate Attributes [1]: [max(csales#73)#76] Results [1]: [max(csales#73)#76 AS tpcds_cmax#77] -Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 -BroadcastExchange (117) -+- * Project (116) - +- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet spark_catalog.default.date_dim (113) +Subquery:4 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 +BroadcastExchange (109) ++- * Project (108) + +- * Filter (107) + +- * ColumnarToRow (106) + +- Scan parquet spark_catalog.default.date_dim (105) -(113) Scan parquet spark_catalog.default.date_dim +(105) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#67, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#67, d_year#78] -(115) Filter [codegen id : 1] +(107) Filter [codegen id : 1] Input [2]: [d_date_sk#67, d_year#78] Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) -(116) Project [codegen id : 1] +(108) Project [codegen id : 1] Output [1]: [d_date_sk#67] Input [2]: [d_date_sk#67, d_year#78] -(117) BroadcastExchange +(109) BroadcastExchange Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] - -Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 5143023afdc62..9c054f1a987d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (18) + WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (17) + WholeStageCodegen (12) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -19,7 +19,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -124,34 +124,18 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Scan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter BroadcastExchange #13 - WholeStageCodegen (15) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (11) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #14 - WholeStageCodegen (10) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - WholeStageCodegen (14) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + WholeStageCodegen (10) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (36) + WholeStageCodegen (26) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] InputAdapter - Exchange [c_last_name,c_first_name] #15 - WholeStageCodegen (35) + Exchange [c_last_name,c_first_name] #14 + WholeStageCodegen (25) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -159,14 +143,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (19) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #16 - WholeStageCodegen (23) + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (18) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] @@ -174,7 +158,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [item_sk] #4 InputAdapter - WholeStageCodegen (27) + WholeStageCodegen (22) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt index fa33257a413c8..cd6474f8dc234 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt @@ -1,114 +1,115 @@ == Physical Plan == -* Sort (110) -+- Exchange (109) - +- * Project (108) - +- * BroadcastHashJoin Inner BuildRight (107) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * HashAggregate (18) - : : : +- Exchange (17) - : : : +- * HashAggregate (16) - : : : +- * Project (15) - : : : +- * SortMergeJoin Inner (14) - : : : :- * Sort (8) - : : : : +- Exchange (7) - : : : : +- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- * Sort (13) - : : : +- Exchange (12) - : : : +- * Filter (11) - : : : +- * ColumnarToRow (10) - : : : +- Scan parquet spark_catalog.default.customer_address (9) - : : +- BroadcastExchange (34) - : : +- * HashAggregate (33) - : : +- Exchange (32) - : : +- * HashAggregate (31) - : : +- * Project (30) - : : +- * SortMergeJoin Inner (29) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.store_sales (19) - : : : +- ReusedExchange (22) - : : +- * Sort (28) - : : +- ReusedExchange (27) - : +- BroadcastExchange (52) - : +- * HashAggregate (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- * Project (48) - : +- * SortMergeJoin Inner (47) - : :- * Sort (44) - : : +- Exchange (43) - : : +- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Filter (39) - : : : +- * ColumnarToRow (38) - : : : +- Scan parquet spark_catalog.default.store_sales (37) - : : +- ReusedExchange (40) - : +- * Sort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (106) - +- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * Project (87) - : +- * BroadcastHashJoin Inner BuildRight (86) - : :- * HashAggregate (69) - : : +- Exchange (68) - : : +- * HashAggregate (67) - : : +- * Project (66) - : : +- * SortMergeJoin Inner (65) - : : :- * Sort (62) - : : : +- Exchange (61) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet spark_catalog.default.web_sales (55) - : : : +- ReusedExchange (58) - : : +- * Sort (64) - : : +- ReusedExchange (63) - : +- BroadcastExchange (85) - : +- * HashAggregate (84) - : +- Exchange (83) - : +- * HashAggregate (82) - : +- * Project (81) - : +- * SortMergeJoin Inner (80) - : :- * Sort (77) - : : +- Exchange (76) - : : +- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * Filter (72) - : : : +- * ColumnarToRow (71) - : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : +- ReusedExchange (73) - : +- * Sort (79) - : +- ReusedExchange (78) - +- BroadcastExchange (103) - +- * HashAggregate (102) - +- Exchange (101) - +- * HashAggregate (100) - +- * Project (99) - +- * SortMergeJoin Inner (98) - :- * Sort (95) - : +- Exchange (94) - : +- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * Filter (90) - : : +- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.web_sales (88) - : +- ReusedExchange (91) - +- * Sort (97) - +- ReusedExchange (96) +* Sort (111) ++- Exchange (110) + +- * Project (109) + +- * Project (108) + +- * BroadcastHashJoin Inner BuildRight (107) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * HashAggregate (18) + : : : +- Exchange (17) + : : : +- * HashAggregate (16) + : : : +- * Project (15) + : : : +- * SortMergeJoin Inner (14) + : : : :- * Sort (8) + : : : : +- Exchange (7) + : : : : +- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- * Sort (13) + : : : +- Exchange (12) + : : : +- * Filter (11) + : : : +- * ColumnarToRow (10) + : : : +- Scan parquet spark_catalog.default.customer_address (9) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * Project (30) + : : +- * SortMergeJoin Inner (29) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Filter (21) + : : : : +- * ColumnarToRow (20) + : : : : +- Scan parquet spark_catalog.default.store_sales (19) + : : : +- ReusedExchange (22) + : : +- * Sort (28) + : : +- ReusedExchange (27) + : +- BroadcastExchange (52) + : +- * HashAggregate (51) + : +- Exchange (50) + : +- * HashAggregate (49) + : +- * Project (48) + : +- * SortMergeJoin Inner (47) + : :- * Sort (44) + : : +- Exchange (43) + : : +- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * Filter (39) + : : : +- * ColumnarToRow (38) + : : : +- Scan parquet spark_catalog.default.store_sales (37) + : : +- ReusedExchange (40) + : +- * Sort (46) + : +- ReusedExchange (45) + +- BroadcastExchange (106) + +- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * HashAggregate (69) + : : +- Exchange (68) + : : +- * HashAggregate (67) + : : +- * Project (66) + : : +- * SortMergeJoin Inner (65) + : : :- * Sort (62) + : : : +- Exchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet spark_catalog.default.web_sales (55) + : : : +- ReusedExchange (58) + : : +- * Sort (64) + : : +- ReusedExchange (63) + : +- BroadcastExchange (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * Project (81) + : +- * SortMergeJoin Inner (80) + : :- * Sort (77) + : : +- Exchange (76) + : : +- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * Filter (72) + : : : +- * ColumnarToRow (71) + : : : +- Scan parquet spark_catalog.default.web_sales (70) + : : +- ReusedExchange (73) + : +- * Sort (79) + : +- ReusedExchange (78) + +- BroadcastExchange (103) + +- * HashAggregate (102) + +- Exchange (101) + +- * HashAggregate (100) + +- * Project (99) + +- * SortMergeJoin Inner (98) + :- * Sort (95) + : +- Exchange (94) + : +- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * Filter (90) + : : +- * ColumnarToRow (89) + : : +- Scan parquet spark_catalog.default.web_sales (88) + : +- ReusedExchange (91) + +- * Sort (97) + +- ReusedExchange (96) (1) Scan parquet spark_catalog.default.store_sales @@ -126,7 +127,7 @@ Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) -(4) ReusedExchange [Reuses operator id: 114] +(4) ReusedExchange [Reuses operator id: 115] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (5) BroadcastHashJoin [codegen id : 2] @@ -212,7 +213,7 @@ Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) -(22) ReusedExchange [Reuses operator id: 118] +(22) ReusedExchange [Reuses operator id: 119] Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] (23) BroadcastHashJoin [codegen id : 8] @@ -297,7 +298,7 @@ Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) -(40) ReusedExchange [Reuses operator id: 122] +(40) ReusedExchange [Reuses operator id: 123] Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] (41) BroadcastHashJoin [codegen id : 15] @@ -382,7 +383,7 @@ Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Condition : isnotnull(ws_bill_addr_sk#38) -(58) ReusedExchange [Reuses operator id: 122] +(58) ReusedExchange [Reuses operator id: 123] Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] (59) BroadcastHashJoin [codegen id : 22] @@ -453,7 +454,7 @@ Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] Condition : isnotnull(ws_bill_addr_sk#50) -(73) ReusedExchange [Reuses operator id: 114] +(73) ReusedExchange [Reuses operator id: 115] Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] (74) BroadcastHashJoin [codegen id : 28] @@ -538,7 +539,7 @@ Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Condition : isnotnull(ws_bill_addr_sk#61) -(91) ReusedExchange [Reuses operator id: 118] +(91) ReusedExchange [Reuses operator id: 119] Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] (92) BroadcastHashJoin [codegen id : 35] @@ -619,91 +620,95 @@ Join type: Inner Join condition: ((CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#37 > 0.00) THEN (store_sales#13 / store_sales#37) END) AND (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END)) (108) Project [codegen id : 42] -Output [6]: [ca_county#34, d_year#31, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#13 / store_sales#37) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#25 / store_sales#13) AS store_q2_q3_increase#75] +Output [8]: [ca_county#34, d_year#31, store_sales#37, store_sales#13, store_sales#25, web_sales#49, web_sales#60, web_sales#71] Input [9]: [store_sales#13, store_sales#25, ca_county#34, d_year#31, store_sales#37, ca_county#45, web_sales#49, web_sales#60, web_sales#71] -(109) Exchange +(109) Project [codegen id : 42] +Output [6]: [ca_county#34, d_year#31, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#13 / store_sales#37) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#25 / store_sales#13) AS store_q2_q3_increase#75] +Input [8]: [ca_county#34, d_year#31, store_sales#37, store_sales#13, store_sales#25, web_sales#49, web_sales#60, web_sales#71] + +(110) Exchange Input [6]: [ca_county#34, d_year#31, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] Arguments: rangepartitioning(ca_county#34 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=19] -(110) Sort [codegen id : 43] +(111) Sort [codegen id : 43] Input [6]: [ca_county#34, d_year#31, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] Arguments: [ca_county#34 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (114) -+- * Filter (113) - +- * ColumnarToRow (112) - +- Scan parquet spark_catalog.default.date_dim (111) +BroadcastExchange (115) ++- * Filter (114) + +- * ColumnarToRow (113) + +- Scan parquet spark_catalog.default.date_dim (112) -(111) Scan parquet spark_catalog.default.date_dim +(112) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(113) Filter [codegen id : 1] +(114) Filter [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(114) BroadcastExchange +(115) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (118) -+- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet spark_catalog.default.date_dim (115) +BroadcastExchange (119) ++- * Filter (118) + +- * ColumnarToRow (117) + +- Scan parquet spark_catalog.default.date_dim (116) -(115) Scan parquet spark_catalog.default.date_dim +(116) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 1] +(117) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(117) Filter [codegen id : 1] +(118) Filter [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 3)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(118) BroadcastExchange +(119) BroadcastExchange Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] Subquery:3 Hosting operator id = 37 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (122) -+- * Filter (121) - +- * ColumnarToRow (120) - +- Scan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (123) ++- * Filter (122) + +- * ColumnarToRow (121) + +- Scan parquet spark_catalog.default.date_dim (120) -(119) Scan parquet spark_catalog.default.date_dim +(120) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] +(121) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(121) Filter [codegen id : 1] +(122) Filter [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 1)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(122) BroadcastExchange +(123) BroadcastExchange Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt index 4a32d17fc293a..1504c1d2a7cf2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt @@ -4,212 +4,213 @@ WholeStageCodegen (43) Exchange [ca_county] #1 WholeStageCodegen (42) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - Project [store_sales,store_sales,ca_county,d_year,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - SortMergeJoin [ss_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_addr_sk] - InputAdapter - Exchange [ss_addr_sk] #3 - WholeStageCodegen (2) - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - InputAdapter - WholeStageCodegen (5) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #5 - WholeStageCodegen (4) - Filter [ca_address_sk,ca_county] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (13) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #7 - WholeStageCodegen (12) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - SortMergeJoin [ss_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ss_addr_sk] - InputAdapter - Exchange [ss_addr_sk] #8 - WholeStageCodegen (8) - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [d_qoy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - InputAdapter - WholeStageCodegen (11) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (20) + Project [ca_county,d_year,store_sales,store_sales,store_sales,web_sales,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + Project [store_sales,store_sales,ca_county,d_year,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #11 - WholeStageCodegen (19) + Exchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (6) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,d_year,d_qoy,ca_county] SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (3) Sort [ss_addr_sk] InputAdapter - Exchange [ss_addr_sk] #12 - WholeStageCodegen (15) + Exchange [ss_addr_sk] #3 + WholeStageCodegen (2) Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_addr_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #13 + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 WholeStageCodegen (1) Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (5) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (41) - Project [ca_county,web_sales,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #15 - WholeStageCodegen (26) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - SortMergeJoin [ws_bill_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (23) - Sort [ws_bill_addr_sk] - InputAdapter - Exchange [ws_bill_addr_sk] #16 - WholeStageCodegen (22) - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - InputAdapter - WholeStageCodegen (25) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (33) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - InputAdapter - Exchange [ca_county,d_qoy,d_year] #18 - WholeStageCodegen (32) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - SortMergeJoin [ws_bill_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (29) - Sort [ws_bill_addr_sk] - InputAdapter - Exchange [ws_bill_addr_sk] #19 - WholeStageCodegen (28) - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - InputAdapter - WholeStageCodegen (31) - Sort [ca_address_sk] + Exchange [ca_address_sk] #5 + WholeStageCodegen (4) + Filter [ca_address_sk,ca_county] + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk,ca_county] #5 + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #20 - WholeStageCodegen (40) + BroadcastExchange #6 + WholeStageCodegen (13) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #7 + WholeStageCodegen (12) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ss_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (9) + Sort [ss_addr_sk] + InputAdapter + Exchange [ss_addr_sk] #8 + WholeStageCodegen (8) + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [d_qoy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + InputAdapter + WholeStageCodegen (11) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #5 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (20) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #11 + WholeStageCodegen (19) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ss_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (16) + Sort [ss_addr_sk] + InputAdapter + Exchange [ss_addr_sk] #12 + WholeStageCodegen (15) + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #13 + WholeStageCodegen (1) + Filter [d_qoy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #13 + InputAdapter + WholeStageCodegen (18) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #5 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (41) + Project [ca_county,web_sales,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county] HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #21 - WholeStageCodegen (39) + Exchange [ca_county,d_qoy,d_year] #15 + WholeStageCodegen (26) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,d_year,d_qoy,ca_county] SortMergeJoin [ws_bill_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (36) + WholeStageCodegen (23) Sort [ws_bill_addr_sk] InputAdapter - Exchange [ws_bill_addr_sk] #22 - WholeStageCodegen (35) + Exchange [ws_bill_addr_sk] #16 + WholeStageCodegen (22) Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_addr_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (25) Sort [ca_address_sk] InputAdapter ReusedExchange [ca_address_sk,ca_county] #5 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (33) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #18 + WholeStageCodegen (32) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ws_bill_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (29) + Sort [ws_bill_addr_sk] + InputAdapter + Exchange [ws_bill_addr_sk] #19 + WholeStageCodegen (28) + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + InputAdapter + WholeStageCodegen (31) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #5 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (40) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #21 + WholeStageCodegen (39) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ws_bill_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (36) + Sort [ws_bill_addr_sk] + InputAdapter + Exchange [ws_bill_addr_sk] #22 + WholeStageCodegen (35) + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + InputAdapter + WholeStageCodegen (38) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index dc2c0e50fa26d..1c3220a4c7037 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -96,7 +96,7 @@ Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, Output [2]: [i_item_sk#7, i_manufact_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_manufact_id), IsNotNull(i_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] @@ -104,7 +104,7 @@ Input [2]: [i_item_sk#7, i_manufact_id#8] (9) Filter [codegen id : 3] Input [2]: [i_item_sk#7, i_manufact_id#8] -Condition : (isnotnull(i_item_sk#7) AND isnotnull(i_manufact_id#8)) +Condition : (isnotnull(i_manufact_id#8) AND isnotnull(i_item_sk#7)) (10) Scan parquet spark_catalog.default.item Output [2]: [i_category#9, i_manufact_id#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index ce831cb030223..4970ce1c8292b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #4 WholeStageCodegen (3) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk,i_manufact_id] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index f1dfd00dd1a38..645aaba289e5e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_manufact_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_manufact_id), IsNotNull(i_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,7 +136,7 @@ Input [2]: [i_item_sk#9, i_manufact_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_manufact_id#10)) +Condition : (isnotnull(i_manufact_id#10) AND isnotnull(i_item_sk#9)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_category#11, i_manufact_id#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 3e322bd8c0c07..53ea01a743df5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk,i_manufact_id] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index f68ecc12142bf..6a769d8c5a2d9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -1,65 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (34) - : : +- Exchange (33) - : : +- * Project (32) - : : +- * Filter (31) - : : +- * SortMergeJoin ExistenceJoin(exists#1) (30) - : : :- * SortMergeJoin ExistenceJoin(exists#2) (22) - : : : :- * SortMergeJoin LeftSemi (14) - : : : : :- * Sort (5) - : : : : : +- Exchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (13) - : : : : +- Exchange (12) - : : : : +- * Project (11) - : : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (9) - : : : +- * Sort (21) - : : : +- Exchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet spark_catalog.default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- * Sort (39) - : +- Exchange (38) - : +- * Filter (37) - : +- * ColumnarToRow (36) - : +- Scan parquet spark_catalog.default.customer_address (35) - +- * Sort (48) - +- Exchange (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet spark_catalog.default.customer_demographics (44) +TakeOrderedAndProject (56) ++- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * Project (52) + +- * Filter (51) + +- * SortMergeJoin ExistenceJoin(exists#1) (50) + :- * SortMergeJoin ExistenceJoin(exists#2) (42) + : :- * Sort (34) + : : +- Exchange (33) + : : +- * Project (32) + : : +- * SortMergeJoin Inner (31) + : : :- * Sort (25) + : : : +- Exchange (24) + : : : +- * Project (23) + : : : +- * SortMergeJoin Inner (22) + : : : :- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * SortMergeJoin LeftSemi (14) + : : : : :- * Sort (5) + : : : : : +- Exchange (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.customer (1) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet spark_catalog.default.customer_address (17) + : : +- * Sort (30) + : : +- Exchange (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet spark_catalog.default.customer_demographics (26) + : +- * Sort (41) + : +- Exchange (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * ColumnarToRow (36) + : : +- Scan parquet spark_catalog.default.web_sales (35) + : +- ReusedExchange (37) + +- * Sort (49) + +- Exchange (48) + +- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * ColumnarToRow (44) + : +- Scan parquet spark_catalog.default.catalog_sales (43) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -67,7 +69,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : ((isnotnull(c_customer_sk#3) AND isnotnull(c_current_addr_sk#5)) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -92,7 +94,7 @@ Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Condition : isnotnull(ss_customer_sk#6) -(9) ReusedExchange [Reuses operator id: 59] +(9) ReusedExchange [Reuses operator id: 61] Output [1]: [d_date_sk#9] (10) BroadcastHashJoin [codegen id : 4] @@ -119,224 +121,232 @@ Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(15) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct +(15) Exchange +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(16) ColumnarToRow [codegen id : 8] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(16) Sort [codegen id : 7] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(17) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#12] +(17) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#10, ca_state#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(18) ColumnarToRow [codegen id : 8] +Input [2]: [ca_address_sk#10, ca_state#11] -(19) Project [codegen id : 8] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] +(19) Filter [codegen id : 8] +Input [2]: [ca_address_sk#10, ca_state#11] +Condition : isnotnull(ca_address_sk#10) (20) Exchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: hashpartitioning(ws_bill_customer_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] (21) Sort [codegen id : 9] -Input [1]: [ws_bill_customer_sk#10] -Arguments: [ws_bill_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 (22) SortMergeJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#10] +Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct +(23) Project [codegen id : 10] +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#10, ca_state#11] -(24) ColumnarToRow [codegen id : 12] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(24) Exchange +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(25) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#15] +(25) Sort [codegen id : 11] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(26) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(26) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(27) Project [codegen id : 12] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +(27) ColumnarToRow [codegen id : 12] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] -(28) Exchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: hashpartitioning(cs_ship_customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(28) Filter [codegen id : 12] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Condition : isnotnull(cd_demo_sk#12) -(29) Sort [codegen id : 13] -Input [1]: [cs_ship_customer_sk#13] -Arguments: [cs_ship_customer_sk#13 ASC NULLS FIRST], false, 0 +(29) Exchange +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: hashpartitioning(cd_demo_sk#12, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(30) SortMergeJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None +(30) Sort [codegen id : 13] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: [cd_demo_sk#12 ASC NULLS FIRST], false, 0 -(31) Filter [codegen id : 14] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) +(31) SortMergeJoin [codegen id : 14] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#12] +Join type: Inner +Join condition: None (32) Project [codegen id : 14] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] +Output [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11, cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] (33) Exchange -Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: hashpartitioning(c_customer_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=7] (34) Sort [codegen id : 15] -Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: [c_customer_sk#3 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +(35) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 16] -Input [2]: [ca_address_sk#16, ca_state#17] - -(37) Filter [codegen id : 16] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#8)] +ReadSchema: struct -(38) Exchange -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: hashpartitioning(ca_address_sk#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) ColumnarToRow [codegen id : 17] +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(39) Sort [codegen id : 17] -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16 ASC NULLS FIRST], false, 0 +(37) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#20] -(40) SortMergeJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] +(38) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 18] -Output [2]: [c_current_cdemo_sk#4, ca_state#17] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] +(39) Project [codegen id : 17] +Output [1]: [ws_bill_customer_sk#18] +Input [3]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20] -(42) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#17] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(40) Exchange +Input [1]: [ws_bill_customer_sk#18] +Arguments: hashpartitioning(ws_bill_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(43) Sort [codegen id : 19] -Input [2]: [c_current_cdemo_sk#4, ca_state#17] -Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 18] +Input [1]: [ws_bill_customer_sk#18] +Arguments: [ws_bill_customer_sk#18 ASC NULLS FIRST], false, 0 + +(42) SortMergeJoin [codegen id : 19] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ws_bill_customer_sk#18] +Join type: ExistenceJoin(exists#2) +Join condition: None -(44) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(43) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#8)] +ReadSchema: struct -(45) ColumnarToRow [codegen id : 20] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(44) ColumnarToRow [codegen id : 21] +Input [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] -(46) Filter [codegen id : 20] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Condition : isnotnull(cd_demo_sk#18) +(45) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#23] -(47) Exchange -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: hashpartitioning(cd_demo_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [cs_sold_date_sk#22] +Right keys [1]: [d_date_sk#23] +Join type: Inner +Join condition: None -(48) Sort [codegen id : 21] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: [cd_demo_sk#18 ASC NULLS FIRST], false, 0 +(47) Project [codegen id : 21] +Output [1]: [cs_ship_customer_sk#21] +Input [3]: [cs_ship_customer_sk#21, cs_sold_date_sk#22, d_date_sk#23] -(49) SortMergeJoin [codegen id : 22] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner +(48) Exchange +Input [1]: [cs_ship_customer_sk#21] +Arguments: hashpartitioning(cs_ship_customer_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(49) Sort [codegen id : 22] +Input [1]: [cs_ship_customer_sk#21] +Arguments: [cs_ship_customer_sk#21 ASC NULLS FIRST], false, 0 + +(50) SortMergeJoin [codegen id : 23] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#21] +Join type: ExistenceJoin(exists#1) Join condition: None -(50) Project [codegen id : 22] -Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(51) Filter [codegen id : 23] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] +Condition : (exists#2 OR exists#1) + +(52) Project [codegen id : 23] +Output [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] -(51) HashAggregate [codegen id : 22] -Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#21), partial_max(cd_dep_count#21), partial_avg(cd_dep_count#21), partial_min(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_avg(cd_dep_employed_count#22), partial_min(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_avg(cd_dep_college_count#23)] +(53) HashAggregate [codegen id : 23] +Input [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#15), partial_max(cd_dep_count#15), partial_avg(cd_dep_count#15), partial_min(cd_dep_employed_count#16), partial_max(cd_dep_employed_count#16), partial_avg(cd_dep_employed_count#16), partial_min(cd_dep_college_count#17), partial_max(cd_dep_college_count#17), partial_avg(cd_dep_college_count#17)] Aggregate Attributes [13]: [count#24, min#25, max#26, sum#27, count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36] -Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] +Results [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -(52) Exchange -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(54) Exchange +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] +Arguments: hashpartitioning(ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(53) HashAggregate [codegen id : 23] -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [count(1), min(cd_dep_count#21), max(cd_dep_count#21), avg(cd_dep_count#21), min(cd_dep_employed_count#22), max(cd_dep_employed_count#22), avg(cd_dep_employed_count#22), min(cd_dep_college_count#23), max(cd_dep_college_count#23), avg(cd_dep_college_count#23)] -Aggregate Attributes [10]: [count(1)#50, min(cd_dep_count#21)#51, max(cd_dep_count#21)#52, avg(cd_dep_count#21)#53, min(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, avg(cd_dep_employed_count#22)#56, min(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, avg(cd_dep_college_count#23)#59] -Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, count(1)#50 AS cnt1#60, min(cd_dep_count#21)#51 AS min(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, avg(cd_dep_count#21)#53 AS avg(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, min(cd_dep_employed_count#22)#54 AS min(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, avg(cd_dep_employed_count#22)#56 AS avg(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, min(cd_dep_college_count#23)#57 AS min(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, avg(cd_dep_college_count#23)#59 AS avg(cd_dep_college_count)#71, cd_dep_count#21] +(55) HashAggregate [codegen id : 24] +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [count(1), min(cd_dep_count#15), max(cd_dep_count#15), avg(cd_dep_count#15), min(cd_dep_employed_count#16), max(cd_dep_employed_count#16), avg(cd_dep_employed_count#16), min(cd_dep_college_count#17), max(cd_dep_college_count#17), avg(cd_dep_college_count#17)] +Aggregate Attributes [10]: [count(1)#50, min(cd_dep_count#15)#51, max(cd_dep_count#15)#52, avg(cd_dep_count#15)#53, min(cd_dep_employed_count#16)#54, max(cd_dep_employed_count#16)#55, avg(cd_dep_employed_count#16)#56, min(cd_dep_college_count#17)#57, max(cd_dep_college_count#17)#58, avg(cd_dep_college_count#17)#59] +Results [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, count(1)#50 AS cnt1#60, min(cd_dep_count#15)#51 AS min(cd_dep_count)#61, max(cd_dep_count#15)#52 AS max(cd_dep_count)#62, avg(cd_dep_count#15)#53 AS avg(cd_dep_count)#63, cd_dep_employed_count#16, count(1)#50 AS cnt2#64, min(cd_dep_employed_count#16)#54 AS min(cd_dep_employed_count)#65, max(cd_dep_employed_count#16)#55 AS max(cd_dep_employed_count)#66, avg(cd_dep_employed_count#16)#56 AS avg(cd_dep_employed_count)#67, cd_dep_college_count#17, count(1)#50 AS cnt3#68, min(cd_dep_college_count#17)#57 AS min(cd_dep_college_count)#69, max(cd_dep_college_count#17)#58 AS max(cd_dep_college_count)#70, avg(cd_dep_college_count#17)#59 AS avg(cd_dep_college_count)#71, cd_dep_count#15] -(54) TakeOrderedAndProject -Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71, cd_dep_count#21] -Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71] +(56) TakeOrderedAndProject +Input [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71, cd_dep_count#15] +Arguments: 100, [ca_state#11 ASC NULLS FIRST, cd_gender#13 ASC NULLS FIRST, cd_marital_status#14 ASC NULLS FIRST, cd_dep_count#15 ASC NULLS FIRST, cd_dep_employed_count#16 ASC NULLS FIRST, cd_dep_college_count#17 ASC NULLS FIRST], [ca_state#11, cd_gender#13, cd_marital_status#14, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet spark_catalog.default.date_dim (55) +BroadcastExchange (61) ++- * Project (60) + +- * Filter (59) + +- * ColumnarToRow (58) + +- Scan parquet spark_catalog.default.date_dim (57) -(55) Scan parquet spark_catalog.default.date_dim +(57) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] +(58) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(57) Filter [codegen id : 1] +(59) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 2002)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#9)) -(58) Project [codegen id : 1] +(60) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(59) BroadcastExchange +(61) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 35 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 6ce4f6e1a81f3..b33ff002c3c70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -1,112 +1,116 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (23) + WholeStageCodegen (24) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (22) + WholeStageCodegen (23) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (19) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (18) - Project [c_current_cdemo_sk,ca_state] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (15) - Sort [c_current_addr_sk] - InputAdapter - Exchange [c_current_addr_sk] #3 - WholeStageCodegen (14) - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - SortMergeJoin [c_customer_sk,cs_ship_customer_sk] - InputAdapter - WholeStageCodegen (10) - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Filter [exists,exists] + SortMergeJoin [c_customer_sk,cs_ship_customer_sk] + InputAdapter + WholeStageCodegen (19) + SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + InputAdapter + WholeStageCodegen (15) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #2 + WholeStageCodegen (14) + Project [c_customer_sk,ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #3 + WholeStageCodegen (10) + Project [c_customer_sk,c_current_cdemo_sk,ca_state] + SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (6) - SortMergeJoin [c_customer_sk,ss_customer_sk] - InputAdapter - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #4 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + WholeStageCodegen (7) + Sort [c_current_addr_sk] InputAdapter - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #5 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #6 + Exchange [c_current_addr_sk] #4 + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #5 + WholeStageCodegen (1) + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #6 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter WholeStageCodegen (9) - Sort [ws_bill_customer_sk] + Sort [ca_address_sk] InputAdapter - Exchange [ws_bill_customer_sk] #7 + Exchange [ca_address_sk] #8 WholeStageCodegen (8) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ca_address_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (13) - Sort [cs_ship_customer_sk] + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + WholeStageCodegen (13) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #9 + WholeStageCodegen (12) + Filter [cd_demo_sk] + ColumnarToRow InputAdapter - Exchange [cs_ship_customer_sk] #8 - WholeStageCodegen (12) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + InputAdapter + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #10 WholeStageCodegen (17) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #9 - WholeStageCodegen (16) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - WholeStageCodegen (21) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #10 - WholeStageCodegen (20) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + WholeStageCodegen (22) + Sort [cs_ship_customer_sk] + InputAdapter + Exchange [cs_ship_customer_sk] #11 + WholeStageCodegen (21) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 866cfa7b6117a..616b5ecc42e71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -4,51 +4,51 @@ TakeOrderedAndProject (43) +- Exchange (41) +- * HashAggregate (40) +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * Filter (26) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (25) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (18) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (7) - : : : +- BroadcastExchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * ColumnarToRow (13) - : : : : +- Scan parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.customer_demographics (34) + +- * Filter (38) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (37) + :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (30) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (15) + : : : +- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet spark_catalog.default.customer_address (12) + : : +- BroadcastExchange (21) + : : +- * Filter (20) + : : +- * ColumnarToRow (19) + : : +- Scan parquet spark_catalog.default.customer_demographics (18) + : +- BroadcastExchange (29) + : +- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * ColumnarToRow (25) + : : +- Scan parquet spark_catalog.default.web_sales (24) + : +- ReusedExchange (26) + +- BroadcastExchange (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (32) + : +- Scan parquet spark_catalog.default.catalog_sales (31) + +- ReusedExchange (33) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -56,7 +56,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : ((isnotnull(c_customer_sk#3) AND isnotnull(c_current_addr_sk#5)) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] @@ -96,157 +96,157 @@ Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(12) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(12) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#10, ca_state#11] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct + +(13) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#10, ca_state#11] -(13) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) Filter [codegen id : 3] +Input [2]: [ca_address_sk#10, ca_state#11] +Condition : isnotnull(ca_address_sk#10) -(14) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#12] +(15) BroadcastExchange +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] +(16) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] +(17) Project [codegen id : 9] +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#10, ca_state#11] -(17) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) +(19) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] + +(20) Filter [codegen id : 4] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Condition : isnotnull(cd_demo_sk#12) + +(21) BroadcastExchange +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#12] +Join type: Inner Join condition: None -(19) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(23) Project [codegen id : 9] +Output [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11, cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] + +(24) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#8)] +ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(25) ColumnarToRow [codegen id : 6] +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(21) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#15] +(26) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#20] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +(28) Project [codegen id : 6] +Output [1]: [ws_bill_customer_sk#18] +Input [3]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20] -(24) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(29) BroadcastExchange +Input [1]: [ws_bill_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) +Right keys [1]: [ws_bill_customer_sk#18] +Join type: ExistenceJoin(exists#2) Join condition: None -(26) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(27) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(28) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +(31) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#17] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#8)] +ReadSchema: struct -(30) Filter [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) +(32) ColumnarToRow [codegen id : 8] +Input [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] -(31) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(33) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#23] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#22] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#17] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] +(35) Project [codegen id : 8] +Output [1]: [cs_ship_customer_sk#21] +Input [3]: [cs_ship_customer_sk#21, cs_sold_date_sk#22, d_date_sk#23] -(34) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(36) BroadcastExchange +Input [1]: [cs_ship_customer_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Condition : isnotnull(cd_demo_sk#18) - -(37) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#21] +Join type: ExistenceJoin(exists#1) Join condition: None +(38) Filter [codegen id : 9] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] +Condition : (exists#2 OR exists#1) + (39) Project [codegen id : 9] -Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +Output [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] (40) HashAggregate [codegen id : 9] -Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#21), partial_max(cd_dep_count#21), partial_avg(cd_dep_count#21), partial_min(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_avg(cd_dep_employed_count#22), partial_min(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_avg(cd_dep_college_count#23)] +Input [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#15), partial_max(cd_dep_count#15), partial_avg(cd_dep_count#15), partial_min(cd_dep_employed_count#16), partial_max(cd_dep_employed_count#16), partial_avg(cd_dep_employed_count#16), partial_min(cd_dep_college_count#17), partial_max(cd_dep_college_count#17), partial_avg(cd_dep_college_count#17)] Aggregate Attributes [13]: [count#24, min#25, max#26, sum#27, count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36] -Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] +Results [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] (41) Exchange -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] +Arguments: hashpartitioning(ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, 5), ENSURE_REQUIREMENTS, [plan_id=6] (42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [count(1), min(cd_dep_count#21), max(cd_dep_count#21), avg(cd_dep_count#21), min(cd_dep_employed_count#22), max(cd_dep_employed_count#22), avg(cd_dep_employed_count#22), min(cd_dep_college_count#23), max(cd_dep_college_count#23), avg(cd_dep_college_count#23)] -Aggregate Attributes [10]: [count(1)#50, min(cd_dep_count#21)#51, max(cd_dep_count#21)#52, avg(cd_dep_count#21)#53, min(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, avg(cd_dep_employed_count#22)#56, min(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, avg(cd_dep_college_count#23)#59] -Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, count(1)#50 AS cnt1#60, min(cd_dep_count#21)#51 AS min(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, avg(cd_dep_count#21)#53 AS avg(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, min(cd_dep_employed_count#22)#54 AS min(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, avg(cd_dep_employed_count#22)#56 AS avg(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, min(cd_dep_college_count#23)#57 AS min(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, avg(cd_dep_college_count#23)#59 AS avg(cd_dep_college_count)#71, cd_dep_count#21] +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [count(1), min(cd_dep_count#15), max(cd_dep_count#15), avg(cd_dep_count#15), min(cd_dep_employed_count#16), max(cd_dep_employed_count#16), avg(cd_dep_employed_count#16), min(cd_dep_college_count#17), max(cd_dep_college_count#17), avg(cd_dep_college_count#17)] +Aggregate Attributes [10]: [count(1)#50, min(cd_dep_count#15)#51, max(cd_dep_count#15)#52, avg(cd_dep_count#15)#53, min(cd_dep_employed_count#16)#54, max(cd_dep_employed_count#16)#55, avg(cd_dep_employed_count#16)#56, min(cd_dep_college_count#17)#57, max(cd_dep_college_count#17)#58, avg(cd_dep_college_count#17)#59] +Results [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, count(1)#50 AS cnt1#60, min(cd_dep_count#15)#51 AS min(cd_dep_count)#61, max(cd_dep_count#15)#52 AS max(cd_dep_count)#62, avg(cd_dep_count#15)#53 AS avg(cd_dep_count)#63, cd_dep_employed_count#16, count(1)#50 AS cnt2#64, min(cd_dep_employed_count#16)#54 AS min(cd_dep_employed_count)#65, max(cd_dep_employed_count#16)#55 AS max(cd_dep_employed_count)#66, avg(cd_dep_employed_count#16)#56 AS avg(cd_dep_employed_count)#67, cd_dep_college_count#17, count(1)#50 AS cnt3#68, min(cd_dep_college_count#17)#57 AS min(cd_dep_college_count)#69, max(cd_dep_college_count#17)#58 AS max(cd_dep_college_count)#70, avg(cd_dep_college_count#17)#59 AS avg(cd_dep_college_count)#71, cd_dep_count#15] (43) TakeOrderedAndProject -Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71, cd_dep_count#21] -Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71] +Input [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71, cd_dep_count#15] +Arguments: 100, [ca_state#11 ASC NULLS FIRST, cd_gender#13 ASC NULLS FIRST, cd_marital_status#14 ASC NULLS FIRST, cd_dep_count#15 ASC NULLS FIRST, cd_dep_employed_count#16 ASC NULLS FIRST, cd_dep_college_count#17 ASC NULLS FIRST], [ca_state#11, cd_gender#13, cd_marital_status#14, cnt1#60, min(cd_dep_count)#61, max(cd_dep_count)#62, avg(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, min(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, avg(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, min(cd_dep_college_count)#69, max(cd_dep_college_count)#70, avg(cd_dep_college_count)#71] ===== Subqueries ===== @@ -280,8 +280,8 @@ Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 24 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index d85b9a98d2b7d..04245b1f5b9dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (9) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_sk,ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_customer_sk,c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -39,37 +39,37 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + WholeStageCodegen (3) + Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - Filter [ca_address_sk] + WholeStageCodegen (6) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index 30cd93a46bf12..c18469fa22ccc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (118) -+- * Project (117) - +- * SortMergeJoin Inner (116) - :- * Project (98) - : +- * SortMergeJoin Inner (97) - : :- * Project (78) - : : +- * SortMergeJoin Inner (77) - : : :- * Project (59) - : : : +- * SortMergeJoin Inner (58) +TakeOrderedAndProject (122) ++- * Project (121) + +- * SortMergeJoin Inner (120) + :- * Project (101) + : +- * SortMergeJoin Inner (100) + : :- * Project (80) + : : +- * SortMergeJoin Inner (79) + : : :- * Project (60) + : : : +- * SortMergeJoin Inner (59) : : : :- * SortMergeJoin Inner (39) : : : : :- * Sort (21) : : : : : +- Exchange (20) @@ -47,76 +47,80 @@ TakeOrderedAndProject (118) : : : : : +- ReusedExchange (25) : : : : +- * Sort (31) : : : : +- ReusedExchange (30) - : : : +- * Sort (57) - : : : +- Exchange (56) - : : : +- * Filter (55) - : : : +- * HashAggregate (54) - : : : +- Exchange (53) - : : : +- * HashAggregate (52) - : : : +- * Project (51) - : : : +- * SortMergeJoin Inner (50) - : : : :- * Sort (47) - : : : : +- Exchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Filter (42) - : : : : : +- * ColumnarToRow (41) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : +- ReusedExchange (43) - : : : +- * Sort (49) - : : : +- ReusedExchange (48) - : : +- * Sort (76) - : : +- Exchange (75) - : : +- * HashAggregate (74) - : : +- Exchange (73) - : : +- * HashAggregate (72) - : : +- * Project (71) - : : +- * SortMergeJoin Inner (70) - : : :- * Sort (67) - : : : +- Exchange (66) - : : : +- * Project (65) - : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : :- * Filter (62) - : : : : +- * ColumnarToRow (61) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (60) - : : : +- ReusedExchange (63) - : : +- * Sort (69) - : : +- ReusedExchange (68) - : +- * Sort (96) - : +- Exchange (95) - : +- * Filter (94) - : +- * HashAggregate (93) - : +- Exchange (92) - : +- * HashAggregate (91) - : +- * Project (90) - : +- * SortMergeJoin Inner (89) - : :- * Sort (86) - : : +- Exchange (85) - : : +- * Project (84) - : : +- * BroadcastHashJoin Inner BuildRight (83) - : : :- * Filter (81) - : : : +- * ColumnarToRow (80) - : : : +- Scan parquet spark_catalog.default.web_sales (79) - : : +- ReusedExchange (82) - : +- * Sort (88) - : +- ReusedExchange (87) - +- * Sort (115) - +- Exchange (114) - +- * HashAggregate (113) - +- Exchange (112) - +- * HashAggregate (111) - +- * Project (110) - +- * SortMergeJoin Inner (109) - :- * Sort (106) - : +- Exchange (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Filter (101) - : : +- * ColumnarToRow (100) - : : +- Scan parquet spark_catalog.default.web_sales (99) - : +- ReusedExchange (102) - +- * Sort (108) - +- ReusedExchange (107) + : : : +- * Sort (58) + : : : +- Exchange (57) + : : : +- * Project (56) + : : : +- * Filter (55) + : : : +- * HashAggregate (54) + : : : +- Exchange (53) + : : : +- * HashAggregate (52) + : : : +- * Project (51) + : : : +- * SortMergeJoin Inner (50) + : : : :- * Sort (47) + : : : : +- Exchange (46) + : : : : +- * Project (45) + : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : :- * Filter (42) + : : : : : +- * ColumnarToRow (41) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) + : : : : +- ReusedExchange (43) + : : : +- * Sort (49) + : : : +- ReusedExchange (48) + : : +- * Sort (78) + : : +- Exchange (77) + : : +- * Project (76) + : : +- * HashAggregate (75) + : : +- Exchange (74) + : : +- * HashAggregate (73) + : : +- * Project (72) + : : +- * SortMergeJoin Inner (71) + : : :- * Sort (68) + : : : +- Exchange (67) + : : : +- * Project (66) + : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : :- * Filter (63) + : : : : +- * ColumnarToRow (62) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (61) + : : : +- ReusedExchange (64) + : : +- * Sort (70) + : : +- ReusedExchange (69) + : +- * Sort (99) + : +- Exchange (98) + : +- * Project (97) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- * Project (92) + : +- * SortMergeJoin Inner (91) + : :- * Sort (88) + : : +- Exchange (87) + : : +- * Project (86) + : : +- * BroadcastHashJoin Inner BuildRight (85) + : : :- * Filter (83) + : : : +- * ColumnarToRow (82) + : : : +- Scan parquet spark_catalog.default.web_sales (81) + : : +- ReusedExchange (84) + : +- * Sort (90) + : +- ReusedExchange (89) + +- * Sort (119) + +- Exchange (118) + +- * Project (117) + +- * HashAggregate (116) + +- Exchange (115) + +- * HashAggregate (114) + +- * Project (113) + +- * SortMergeJoin Inner (112) + :- * Sort (109) + : +- Exchange (108) + : +- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * Filter (104) + : : +- * ColumnarToRow (103) + : : +- Scan parquet spark_catalog.default.web_sales (102) + : +- ReusedExchange (105) + +- * Sort (111) + +- ReusedExchange (110) (1) Scan parquet spark_catalog.default.store_sales @@ -134,7 +138,7 @@ Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ex Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 122] +(4) ReusedExchange [Reuses operator id: 126] Output [2]: [d_date_sk#8, d_year#9] (5) BroadcastHashJoin [codegen id : 2] @@ -232,7 +236,7 @@ Input [6]: [ss_customer_sk#25, ss_ext_discount_amt#26, ss_ext_sales_price#27, ss Input [6]: [ss_customer_sk#25, ss_ext_discount_amt#26, ss_ext_sales_price#27, ss_ext_wholesale_cost#28, ss_ext_list_price#29, ss_sold_date_sk#30] Condition : isnotnull(ss_customer_sk#25) -(25) ReusedExchange [Reuses operator id: 126] +(25) ReusedExchange [Reuses operator id: 130] Output [2]: [d_date_sk#32, d_year#33] (26) BroadcastHashJoin [codegen id : 10] @@ -317,7 +321,7 @@ Input [6]: [cs_bill_customer_sk#54, cs_ext_discount_amt#55, cs_ext_sales_price#5 Input [6]: [cs_bill_customer_sk#54, cs_ext_discount_amt#55, cs_ext_sales_price#56, cs_ext_wholesale_cost#57, cs_ext_list_price#58, cs_sold_date_sk#59] Condition : isnotnull(cs_bill_customer_sk#54) -(43) ReusedExchange [Reuses operator id: 122] +(43) ReusedExchange [Reuses operator id: 126] Output [2]: [d_date_sk#60, d_year#61] (44) BroadcastHashJoin [codegen id : 19] @@ -377,357 +381,373 @@ Results [2]: [c_customer_id#63 AS customer_id#75, sum(((((cs_ext_list_price#58 - Input [2]: [customer_id#75, year_total#76] Condition : (isnotnull(year_total#76) AND (year_total#76 > 0.000000)) -(56) Exchange +(56) Project [codegen id : 24] +Output [2]: [customer_id#75 AS customer_id#77, year_total#76 AS year_total#78] Input [2]: [customer_id#75, year_total#76] -Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(57) Sort [codegen id : 25] -Input [2]: [customer_id#75, year_total#76] -Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 +(57) Exchange +Input [2]: [customer_id#77, year_total#78] +Arguments: hashpartitioning(customer_id#77, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(58) Sort [codegen id : 25] +Input [2]: [customer_id#77, year_total#78] +Arguments: [customer_id#77 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 26] +(59) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#75] +Right keys [1]: [customer_id#77] Join type: Inner Join condition: None -(59) Project [codegen id : 26] -Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#76] -Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#75, year_total#76] +(60) Project [codegen id : 26] +Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#78] +Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#77, year_total#78] -(60) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, cs_sold_date_sk#82] +(61) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, cs_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#82), dynamicpruningexpression(cs_sold_date_sk#82 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#31)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, cs_sold_date_sk#82] +(62) ColumnarToRow [codegen id : 28] +Input [6]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, cs_sold_date_sk#84] -(62) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, cs_sold_date_sk#82] -Condition : isnotnull(cs_bill_customer_sk#77) +(63) Filter [codegen id : 28] +Input [6]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, cs_sold_date_sk#84] +Condition : isnotnull(cs_bill_customer_sk#79) -(63) ReusedExchange [Reuses operator id: 126] -Output [2]: [d_date_sk#83, d_year#84] +(64) ReusedExchange [Reuses operator id: 130] +Output [2]: [d_date_sk#85, d_year#86] -(64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#82] -Right keys [1]: [d_date_sk#83] +(65) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_sold_date_sk#84] +Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(65) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84] -Input [8]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, cs_sold_date_sk#82, d_date_sk#83, d_year#84] +(66) Project [codegen id : 28] +Output [6]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, d_year#86] +Input [8]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, cs_sold_date_sk#84, d_date_sk#85, d_year#86] -(66) Exchange -Input [6]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84] -Arguments: hashpartitioning(cs_bill_customer_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(67) Exchange +Input [6]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, d_year#86] +Arguments: hashpartitioning(cs_bill_customer_sk#79, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(67) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84] -Arguments: [cs_bill_customer_sk#77 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 29] +Input [6]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, d_year#86] +Arguments: [cs_bill_customer_sk#79 ASC NULLS FIRST], false, 0 -(68) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92] +(69) ReusedExchange [Reuses operator id: 12] +Output [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94] -(69) Sort [codegen id : 31] -Input [8]: [c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92] -Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 31] +Input [8]: [c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94] +Arguments: [c_customer_sk#87 ASC NULLS FIRST], false, 0 -(70) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#77] -Right keys [1]: [c_customer_sk#85] +(71) SortMergeJoin [codegen id : 32] +Left keys [1]: [cs_bill_customer_sk#79] +Right keys [1]: [c_customer_sk#87] Join type: Inner Join condition: None -(71) Project [codegen id : 32] -Output [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84] -Input [14]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84, c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92] - -(72) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84] -Functions [1]: [partial_sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84, sum#95, isEmpty#96] - -(73) Exchange -Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(74) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84, sum#95, isEmpty#96] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84] -Functions [1]: [sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))#74] -Results [2]: [c_customer_id#86 AS customer_id#97, sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))#74 AS year_total#98] - -(75) Exchange -Input [2]: [customer_id#97, year_total#98] -Arguments: hashpartitioning(customer_id#97, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(76) Sort [codegen id : 34] -Input [2]: [customer_id#97, year_total#98] -Arguments: [customer_id#97 ASC NULLS FIRST], false, 0 - -(77) SortMergeJoin [codegen id : 35] +(72) Project [codegen id : 32] +Output [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, d_year#86] +Input [14]: [cs_bill_customer_sk#79, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, d_year#86, c_customer_sk#87, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94] + +(73) HashAggregate [codegen id : 32] +Input [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, cs_ext_discount_amt#80, cs_ext_sales_price#81, cs_ext_wholesale_cost#82, cs_ext_list_price#83, d_year#86] +Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#86] +Functions [1]: [partial_sum(((((cs_ext_list_price#83 - cs_ext_wholesale_cost#82) - cs_ext_discount_amt#80) + cs_ext_sales_price#81) / 2))] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#86, sum#97, isEmpty#98] + +(74) Exchange +Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#86, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#86, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(75) HashAggregate [codegen id : 33] +Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#86, sum#97, isEmpty#98] +Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#92, c_login#93, c_email_address#94, d_year#86] +Functions [1]: [sum(((((cs_ext_list_price#83 - cs_ext_wholesale_cost#82) - cs_ext_discount_amt#80) + cs_ext_sales_price#81) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#83 - cs_ext_wholesale_cost#82) - cs_ext_discount_amt#80) + cs_ext_sales_price#81) / 2))#74] +Results [2]: [c_customer_id#88 AS customer_id#99, sum(((((cs_ext_list_price#83 - cs_ext_wholesale_cost#82) - cs_ext_discount_amt#80) + cs_ext_sales_price#81) / 2))#74 AS year_total#100] + +(76) Project [codegen id : 33] +Output [2]: [customer_id#99 AS customer_id#101, year_total#100 AS year_total#102] +Input [2]: [customer_id#99, year_total#100] + +(77) Exchange +Input [2]: [customer_id#101, year_total#102] +Arguments: hashpartitioning(customer_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(78) Sort [codegen id : 34] +Input [2]: [customer_id#101, year_total#102] +Arguments: [customer_id#101 ASC NULLS FIRST], false, 0 + +(79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#97] +Right keys [1]: [customer_id#101] Join type: Inner -Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) +Join condition: (CASE WHEN (year_total#78 > 0.000000) THEN (year_total#102 / year_total#78) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) -(78) Project [codegen id : 35] -Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98] -Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#76, customer_id#97, year_total#98] +(80) Project [codegen id : 35] +Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102] +Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#78, customer_id#101, year_total#102] -(79) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +(81) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] +(82) ColumnarToRow [codegen id : 37] +Input [6]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, ws_sold_date_sk#108] -(81) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_bill_customer_sk#99) +(83) Filter [codegen id : 37] +Input [6]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_bill_customer_sk#103) -(82) ReusedExchange [Reuses operator id: 122] -Output [2]: [d_date_sk#105, d_year#106] +(84) ReusedExchange [Reuses operator id: 126] +Output [2]: [d_date_sk#109, d_year#110] -(83) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#104] -Right keys [1]: [d_date_sk#105] +(85) BroadcastHashJoin [codegen id : 37] +Left keys [1]: [ws_sold_date_sk#108] +Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(84) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106] -Input [8]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, ws_sold_date_sk#104, d_date_sk#105, d_year#106] +(86) Project [codegen id : 37] +Output [6]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, d_year#110] +Input [8]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, ws_sold_date_sk#108, d_date_sk#109, d_year#110] -(85) Exchange -Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106] -Arguments: hashpartitioning(ws_bill_customer_sk#99, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(87) Exchange +Input [6]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, d_year#110] +Arguments: hashpartitioning(ws_bill_customer_sk#103, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(86) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106] -Arguments: [ws_bill_customer_sk#99 ASC NULLS FIRST], false, 0 +(88) Sort [codegen id : 38] +Input [6]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, d_year#110] +Arguments: [ws_bill_customer_sk#103 ASC NULLS FIRST], false, 0 -(87) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#107, c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114] +(89) ReusedExchange [Reuses operator id: 12] +Output [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -(88) Sort [codegen id : 40] -Input [8]: [c_customer_sk#107, c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114] -Arguments: [c_customer_sk#107 ASC NULLS FIRST], false, 0 +(90) Sort [codegen id : 40] +Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] +Arguments: [c_customer_sk#111 ASC NULLS FIRST], false, 0 -(89) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#99] -Right keys [1]: [c_customer_sk#107] +(91) SortMergeJoin [codegen id : 41] +Left keys [1]: [ws_bill_customer_sk#103] +Right keys [1]: [c_customer_sk#111] Join type: Inner Join condition: None -(90) Project [codegen id : 41] -Output [12]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106] -Input [14]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106, c_customer_sk#107, c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114] - -(91) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106] -Keys [8]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106] -Functions [1]: [partial_sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))] -Aggregate Attributes [2]: [sum#115, isEmpty#116] -Results [10]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106, sum#117, isEmpty#118] - -(92) Exchange -Input [10]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106, sum#117, isEmpty#118] -Arguments: hashpartitioning(c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] - -(93) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106, sum#117, isEmpty#118] -Keys [8]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106] -Functions [1]: [sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))#119] -Results [2]: [c_customer_id#108 AS customer_id#120, sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))#119 AS year_total#121] - -(94) Filter [codegen id : 42] -Input [2]: [customer_id#120, year_total#121] -Condition : (isnotnull(year_total#121) AND (year_total#121 > 0.000000)) - -(95) Exchange -Input [2]: [customer_id#120, year_total#121] -Arguments: hashpartitioning(customer_id#120, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(96) Sort [codegen id : 43] -Input [2]: [customer_id#120, year_total#121] -Arguments: [customer_id#120 ASC NULLS FIRST], false, 0 - -(97) SortMergeJoin [codegen id : 44] +(92) Project [codegen id : 41] +Output [12]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, d_year#110] +Input [14]: [ws_bill_customer_sk#103, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, d_year#110, c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] + +(93) HashAggregate [codegen id : 41] +Input [12]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, ws_ext_discount_amt#104, ws_ext_sales_price#105, ws_ext_wholesale_cost#106, ws_ext_list_price#107, d_year#110] +Keys [8]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, d_year#110] +Functions [1]: [partial_sum(((((ws_ext_list_price#107 - ws_ext_wholesale_cost#106) - ws_ext_discount_amt#104) + ws_ext_sales_price#105) / 2))] +Aggregate Attributes [2]: [sum#119, isEmpty#120] +Results [10]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, d_year#110, sum#121, isEmpty#122] + +(94) Exchange +Input [10]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, d_year#110, sum#121, isEmpty#122] +Arguments: hashpartitioning(c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, d_year#110, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(95) HashAggregate [codegen id : 42] +Input [10]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, d_year#110, sum#121, isEmpty#122] +Keys [8]: [c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118, d_year#110] +Functions [1]: [sum(((((ws_ext_list_price#107 - ws_ext_wholesale_cost#106) - ws_ext_discount_amt#104) + ws_ext_sales_price#105) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#107 - ws_ext_wholesale_cost#106) - ws_ext_discount_amt#104) + ws_ext_sales_price#105) / 2))#123] +Results [2]: [c_customer_id#112 AS customer_id#124, sum(((((ws_ext_list_price#107 - ws_ext_wholesale_cost#106) - ws_ext_discount_amt#104) + ws_ext_sales_price#105) / 2))#123 AS year_total#125] + +(96) Filter [codegen id : 42] +Input [2]: [customer_id#124, year_total#125] +Condition : (isnotnull(year_total#125) AND (year_total#125 > 0.000000)) + +(97) Project [codegen id : 42] +Output [2]: [customer_id#124 AS customer_id#126, year_total#125 AS year_total#127] +Input [2]: [customer_id#124, year_total#125] + +(98) Exchange +Input [2]: [customer_id#126, year_total#127] +Arguments: hashpartitioning(customer_id#126, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(99) Sort [codegen id : 43] +Input [2]: [customer_id#126, year_total#127] +Arguments: [customer_id#126 ASC NULLS FIRST], false, 0 + +(100) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#120] +Right keys [1]: [customer_id#126] Join type: Inner Join condition: None -(98) Project [codegen id : 44] -Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98, year_total#121] -Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98, customer_id#120, year_total#121] +(101) Project [codegen id : 44] +Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102, year_total#127] +Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102, customer_id#126, year_total#127] -(99) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +(102) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, ws_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#127), dynamicpruningexpression(ws_sold_date_sk#127 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#133), dynamicpruningexpression(ws_sold_date_sk#133 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] +(103) ColumnarToRow [codegen id : 46] +Input [6]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, ws_sold_date_sk#133] -(101) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127] -Condition : isnotnull(ws_bill_customer_sk#122) +(104) Filter [codegen id : 46] +Input [6]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, ws_sold_date_sk#133] +Condition : isnotnull(ws_bill_customer_sk#128) -(102) ReusedExchange [Reuses operator id: 126] -Output [2]: [d_date_sk#128, d_year#129] +(105) ReusedExchange [Reuses operator id: 130] +Output [2]: [d_date_sk#134, d_year#135] -(103) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#127] -Right keys [1]: [d_date_sk#128] +(106) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [ws_sold_date_sk#133] +Right keys [1]: [d_date_sk#134] Join type: Inner Join condition: None -(104) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129] -Input [8]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, ws_sold_date_sk#127, d_date_sk#128, d_year#129] +(107) Project [codegen id : 46] +Output [6]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, d_year#135] +Input [8]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, ws_sold_date_sk#133, d_date_sk#134, d_year#135] -(105) Exchange -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129] -Arguments: hashpartitioning(ws_bill_customer_sk#122, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(108) Exchange +Input [6]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, d_year#135] +Arguments: hashpartitioning(ws_bill_customer_sk#128, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(106) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129] -Arguments: [ws_bill_customer_sk#122 ASC NULLS FIRST], false, 0 +(109) Sort [codegen id : 47] +Input [6]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, d_year#135] +Arguments: [ws_bill_customer_sk#128 ASC NULLS FIRST], false, 0 -(107) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] +(110) ReusedExchange [Reuses operator id: 12] +Output [8]: [c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143] -(108) Sort [codegen id : 49] -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Arguments: [c_customer_sk#130 ASC NULLS FIRST], false, 0 +(111) Sort [codegen id : 49] +Input [8]: [c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143] +Arguments: [c_customer_sk#136 ASC NULLS FIRST], false, 0 -(109) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#122] -Right keys [1]: [c_customer_sk#130] +(112) SortMergeJoin [codegen id : 50] +Left keys [1]: [ws_bill_customer_sk#128] +Right keys [1]: [c_customer_sk#136] Join type: Inner Join condition: None -(110) Project [codegen id : 50] -Output [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129] -Input [14]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129, c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] - -(111) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129] -Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129] -Functions [1]: [partial_sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))] -Aggregate Attributes [2]: [sum#138, isEmpty#139] -Results [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129, sum#140, isEmpty#141] - -(112) Exchange -Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129, sum#140, isEmpty#141] -Arguments: hashpartitioning(c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129, 5), ENSURE_REQUIREMENTS, [plan_id=18] - -(113) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129, sum#140, isEmpty#141] -Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129] -Functions [1]: [sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))#119] -Results [2]: [c_customer_id#131 AS customer_id#142, sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))#119 AS year_total#143] - -(114) Exchange -Input [2]: [customer_id#142, year_total#143] -Arguments: hashpartitioning(customer_id#142, 5), ENSURE_REQUIREMENTS, [plan_id=19] - -(115) Sort [codegen id : 52] -Input [2]: [customer_id#142, year_total#143] -Arguments: [customer_id#142 ASC NULLS FIRST], false, 0 - -(116) SortMergeJoin [codegen id : 53] +(113) Project [codegen id : 50] +Output [12]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, d_year#135] +Input [14]: [ws_bill_customer_sk#128, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, d_year#135, c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143] + +(114) HashAggregate [codegen id : 50] +Input [12]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_ext_discount_amt#129, ws_ext_sales_price#130, ws_ext_wholesale_cost#131, ws_ext_list_price#132, d_year#135] +Keys [8]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#135] +Functions [1]: [partial_sum(((((ws_ext_list_price#132 - ws_ext_wholesale_cost#131) - ws_ext_discount_amt#129) + ws_ext_sales_price#130) / 2))] +Aggregate Attributes [2]: [sum#144, isEmpty#145] +Results [10]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#135, sum#146, isEmpty#147] + +(115) Exchange +Input [10]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#135, sum#146, isEmpty#147] +Arguments: hashpartitioning(c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#135, 5), ENSURE_REQUIREMENTS, [plan_id=18] + +(116) HashAggregate [codegen id : 51] +Input [10]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#135, sum#146, isEmpty#147] +Keys [8]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#135] +Functions [1]: [sum(((((ws_ext_list_price#132 - ws_ext_wholesale_cost#131) - ws_ext_discount_amt#129) + ws_ext_sales_price#130) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#132 - ws_ext_wholesale_cost#131) - ws_ext_discount_amt#129) + ws_ext_sales_price#130) / 2))#123] +Results [2]: [c_customer_id#137 AS customer_id#148, sum(((((ws_ext_list_price#132 - ws_ext_wholesale_cost#131) - ws_ext_discount_amt#129) + ws_ext_sales_price#130) / 2))#123 AS year_total#149] + +(117) Project [codegen id : 51] +Output [2]: [customer_id#148 AS customer_id#150, year_total#149 AS year_total#151] +Input [2]: [customer_id#148, year_total#149] + +(118) Exchange +Input [2]: [customer_id#150, year_total#151] +Arguments: hashpartitioning(customer_id#150, 5), ENSURE_REQUIREMENTS, [plan_id=19] + +(119) Sort [codegen id : 52] +Input [2]: [customer_id#150, year_total#151] +Arguments: [customer_id#150 ASC NULLS FIRST], false, 0 + +(120) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#142] +Right keys [1]: [customer_id#150] Join type: Inner -Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#121 > 0.000000) THEN (year_total#143 / year_total#121) END) +Join condition: (CASE WHEN (year_total#78 > 0.000000) THEN (year_total#102 / year_total#78) END > CASE WHEN (year_total#127 > 0.000000) THEN (year_total#151 / year_total#127) END) -(117) Project [codegen id : 53] +(121) Project [codegen id : 53] Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] -Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98, year_total#121, customer_id#142, year_total#143] +Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102, year_total#127, customer_id#150, year_total#151] -(118) TakeOrderedAndProject +(122) TakeOrderedAndProject Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (122) -+- * Filter (121) - +- * ColumnarToRow (120) - +- Scan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (126) ++- * Filter (125) + +- * ColumnarToRow (124) + +- Scan parquet spark_catalog.default.date_dim (123) -(119) Scan parquet spark_catalog.default.date_dim +(123) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_year#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#8, d_year#9] -(121) Filter [codegen id : 1] +(125) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_year#9] Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2001)) AND isnotnull(d_date_sk#8)) -(122) BroadcastExchange +(126) BroadcastExchange Input [2]: [d_date_sk#8, d_year#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 -BroadcastExchange (126) -+- * Filter (125) - +- * ColumnarToRow (124) - +- Scan parquet spark_catalog.default.date_dim (123) +BroadcastExchange (130) ++- * Filter (129) + +- * ColumnarToRow (128) + +- Scan parquet spark_catalog.default.date_dim (127) -(123) Scan parquet spark_catalog.default.date_dim +(127) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(124) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(125) Filter [codegen id : 1] +(129) Filter [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2002)) AND isnotnull(d_date_sk#32)) -(126) BroadcastExchange +(130) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 60 Hosting Expression = cs_sold_date_sk#82 IN dynamicpruning#31 +Subquery:4 Hosting operator id = 61 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#31 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#127 IN dynamicpruning#31 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#133 IN dynamicpruning#31 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt index 0d938e2a03fa5..4afeb995bba9c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt @@ -107,131 +107,135 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter Exchange [customer_id] #10 WholeStageCodegen (24) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (20) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #12 - WholeStageCodegen (19) - Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #4 - InputAdapter - WholeStageCodegen (22) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (23) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - WholeStageCodegen (32) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #15 - WholeStageCodegen (28) - Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #9 - InputAdapter - WholeStageCodegen (31) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (32) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #15 + WholeStageCodegen (28) + Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #9 + InputAdapter + WholeStageCodegen (31) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (43) Sort [customer_id] InputAdapter Exchange [customer_id] #16 WholeStageCodegen (42) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - WholeStageCodegen (41) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (38) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #18 - WholeStageCodegen (37) - Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #4 - InputAdapter - WholeStageCodegen (40) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 + WholeStageCodegen (41) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (38) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #18 + WholeStageCodegen (37) + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + WholeStageCodegen (40) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (52) Sort [customer_id] InputAdapter Exchange [customer_id] #19 WholeStageCodegen (51) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - WholeStageCodegen (50) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (47) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #21 - WholeStageCodegen (46) - Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #9 - InputAdapter - WholeStageCodegen (49) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 + WholeStageCodegen (50) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (47) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #21 + WholeStageCodegen (46) + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #9 + InputAdapter + WholeStageCodegen (49) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index a71aed58a4834..cb5eaf4ec1617 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (108) -+- * Project (107) - +- * BroadcastHashJoin Inner BuildRight (106) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) +TakeOrderedAndProject (112) ++- * Project (111) + +- * BroadcastHashJoin Inner BuildRight (110) + :- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * Project (72) + : : +- * BroadcastHashJoin Inner BuildRight (71) + : : :- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : :- * BroadcastHashJoin Inner BuildRight (33) : : : : :- * Filter (16) : : : : : +- * HashAggregate (15) @@ -41,72 +41,76 @@ TakeOrderedAndProject (108) : : : : : +- * ColumnarToRow (21) : : : : : +- Scan parquet spark_catalog.default.store_sales (20) : : : : +- ReusedExchange (26) - : : : +- BroadcastExchange (50) - : : : +- * Filter (49) - : : : +- * HashAggregate (48) - : : : +- Exchange (47) - : : : +- * HashAggregate (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (36) - : : : : : +- * ColumnarToRow (35) - : : : : : +- Scan parquet spark_catalog.default.customer (34) - : : : : +- BroadcastExchange (40) - : : : : +- * Filter (39) - : : : : +- * ColumnarToRow (38) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (37) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (68) - : : +- * HashAggregate (67) - : : +- Exchange (66) - : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * Filter (55) - : : : : +- * ColumnarToRow (54) - : : : : +- Scan parquet spark_catalog.default.customer (53) - : : : +- BroadcastExchange (59) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet spark_catalog.default.catalog_sales (56) - : : +- ReusedExchange (62) - : +- BroadcastExchange (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Filter (73) - : : : +- * ColumnarToRow (72) - : : : +- Scan parquet spark_catalog.default.customer (71) - : : +- BroadcastExchange (77) - : : +- * Filter (76) - : : +- * ColumnarToRow (75) - : : +- Scan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (80) - +- BroadcastExchange (105) - +- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Project (101) - +- * BroadcastHashJoin Inner BuildRight (100) - :- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Filter (92) - : : +- * ColumnarToRow (91) - : : +- Scan parquet spark_catalog.default.customer (90) - : +- BroadcastExchange (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet spark_catalog.default.web_sales (93) - +- ReusedExchange (99) + : : : +- BroadcastExchange (51) + : : : +- * Project (50) + : : : +- * Filter (49) + : : : +- * HashAggregate (48) + : : : +- Exchange (47) + : : : +- * HashAggregate (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (36) + : : : : : +- * ColumnarToRow (35) + : : : : : +- Scan parquet spark_catalog.default.customer (34) + : : : : +- BroadcastExchange (40) + : : : : +- * Filter (39) + : : : : +- * ColumnarToRow (38) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (37) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (70) + : : +- * Project (69) + : : +- * HashAggregate (68) + : : +- Exchange (67) + : : +- * HashAggregate (66) + : : +- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Project (62) + : : : +- * BroadcastHashJoin Inner BuildRight (61) + : : : :- * Filter (56) + : : : : +- * ColumnarToRow (55) + : : : : +- Scan parquet spark_catalog.default.customer (54) + : : : +- BroadcastExchange (60) + : : : +- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet spark_catalog.default.catalog_sales (57) + : : +- ReusedExchange (63) + : +- BroadcastExchange (90) + : +- * Project (89) + : +- * Filter (88) + : +- * HashAggregate (87) + : +- Exchange (86) + : +- * HashAggregate (85) + : +- * Project (84) + : +- * BroadcastHashJoin Inner BuildRight (83) + : :- * Project (81) + : : +- * BroadcastHashJoin Inner BuildRight (80) + : : :- * Filter (75) + : : : +- * ColumnarToRow (74) + : : : +- Scan parquet spark_catalog.default.customer (73) + : : +- BroadcastExchange (79) + : : +- * Filter (78) + : : +- * ColumnarToRow (77) + : : +- Scan parquet spark_catalog.default.web_sales (76) + : +- ReusedExchange (82) + +- BroadcastExchange (109) + +- * Project (108) + +- * HashAggregate (107) + +- Exchange (106) + +- * HashAggregate (105) + +- * Project (104) + +- * BroadcastHashJoin Inner BuildRight (103) + :- * Project (101) + : +- * BroadcastHashJoin Inner BuildRight (100) + : :- * Filter (95) + : : +- * ColumnarToRow (94) + : : +- Scan parquet spark_catalog.default.customer (93) + : +- BroadcastExchange (99) + : +- * Filter (98) + : +- * ColumnarToRow (97) + : +- Scan parquet spark_catalog.default.web_sales (96) + +- ReusedExchange (102) (1) Scan parquet spark_catalog.default.customer @@ -152,7 +156,7 @@ Join condition: None Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(10) ReusedExchange [Reuses operator id: 112] +(10) ReusedExchange [Reuses operator id: 116] Output [2]: [d_date_sk#16, d_year#17] (11) BroadcastHashJoin [codegen id : 3] @@ -230,7 +234,7 @@ Join condition: None Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Input [14]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(26) ReusedExchange [Reuses operator id: 116] +(26) ReusedExchange [Reuses operator id: 120] Output [2]: [d_date_sk#40, d_year#41] (27) BroadcastHashJoin [codegen id : 6] @@ -314,7 +318,7 @@ Join condition: None Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -(43) ReusedExchange [Reuses operator id: 112] +(43) ReusedExchange [Reuses operator id: 116] Output [2]: [d_date_sk#68, d_year#69] (44) BroadcastHashJoin [codegen id : 10] @@ -349,350 +353,366 @@ Results [2]: [c_customer_id#55 AS customer_id#75, sum(((((cs_ext_list_price#66 - Input [2]: [customer_id#75, year_total#76] Condition : (isnotnull(year_total#76) AND (year_total#76 > 0.000000)) -(50) BroadcastExchange +(50) Project [codegen id : 11] +Output [2]: [customer_id#75 AS customer_id#77, year_total#76 AS year_total#78] Input [2]: [customer_id#75, year_total#76] + +(51) BroadcastExchange +Input [2]: [customer_id#77, year_total#78] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 24] +(52) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#75] +Right keys [1]: [customer_id#77] Join type: Inner Join condition: None -(52) Project [codegen id : 24] -Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#76] -Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#75, year_total#76] +(53) Project [codegen id : 24] +Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#78] +Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#77, year_total#78] -(53) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84] +(54) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84] +(55) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86] -(55) Filter [codegen id : 14] -Input [8]: [c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84] -Condition : (isnotnull(c_customer_sk#77) AND isnotnull(c_customer_id#78)) +(56) Filter [codegen id : 14] +Input [8]: [c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86] +Condition : (isnotnull(c_customer_sk#79) AND isnotnull(c_customer_id#80)) -(56) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#85, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90] +(57) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#90), dynamicpruningexpression(cs_sold_date_sk#90 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#92), dynamicpruningexpression(cs_sold_date_sk#92 IN dynamicpruning#39)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#85, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90] +(58) ColumnarToRow [codegen id : 12] +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] -(58) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#85, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90] -Condition : isnotnull(cs_bill_customer_sk#85) +(59) Filter [codegen id : 12] +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] +Condition : isnotnull(cs_bill_customer_sk#87) -(59) BroadcastExchange -Input [6]: [cs_bill_customer_sk#85, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90] +(60) BroadcastExchange +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#77] -Right keys [1]: [cs_bill_customer_sk#85] +(61) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#79] +Right keys [1]: [cs_bill_customer_sk#87] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [12]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90] -Input [14]: [c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, cs_bill_customer_sk#85, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90] +(62) Project [codegen id : 14] +Output [12]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] +Input [14]: [c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] -(62) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#91, d_year#92] +(63) ReusedExchange [Reuses operator id: 120] +Output [2]: [d_date_sk#93, d_year#94] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#90] -Right keys [1]: [d_date_sk#91] +(64) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [cs_sold_date_sk#92] +Right keys [1]: [d_date_sk#93] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [12]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, d_year#92] -Input [14]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, cs_sold_date_sk#90, d_date_sk#91, d_year#92] - -(65) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, d_year#92] -Keys [8]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92] -Functions [1]: [partial_sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92, sum#95, isEmpty#96] - -(66) Exchange -Input [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92, sum#95, isEmpty#96] -Keys [8]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92] -Functions [1]: [sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))#74] -Results [2]: [c_customer_id#78 AS customer_id#97, sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))#74 AS year_total#98] - -(68) BroadcastExchange -Input [2]: [customer_id#97, year_total#98] +(65) Project [codegen id : 14] +Output [12]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Input [14]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92, d_date_sk#93, d_year#94] + +(66) HashAggregate [codegen id : 14] +Input [12]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Keys [8]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#94] +Functions [1]: [partial_sum(((((cs_ext_list_price#91 - cs_ext_wholesale_cost#90) - cs_ext_discount_amt#88) + cs_ext_sales_price#89) / 2))] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [10]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#94, sum#97, isEmpty#98] + +(67) Exchange +Input [10]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#94, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#94, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(68) HashAggregate [codegen id : 15] +Input [10]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#94, sum#97, isEmpty#98] +Keys [8]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#94] +Functions [1]: [sum(((((cs_ext_list_price#91 - cs_ext_wholesale_cost#90) - cs_ext_discount_amt#88) + cs_ext_sales_price#89) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#91 - cs_ext_wholesale_cost#90) - cs_ext_discount_amt#88) + cs_ext_sales_price#89) / 2))#74] +Results [2]: [c_customer_id#80 AS customer_id#99, sum(((((cs_ext_list_price#91 - cs_ext_wholesale_cost#90) - cs_ext_discount_amt#88) + cs_ext_sales_price#89) / 2))#74 AS year_total#100] + +(69) Project [codegen id : 15] +Output [2]: [customer_id#99 AS customer_id#101, year_total#100 AS year_total#102] +Input [2]: [customer_id#99, year_total#100] + +(70) BroadcastExchange +Input [2]: [customer_id#101, year_total#102] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 24] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#97] +Right keys [1]: [customer_id#101] Join type: Inner -Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) +Join condition: (CASE WHEN (year_total#78 > 0.000000) THEN (year_total#102 / year_total#78) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) -(70) Project [codegen id : 24] -Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98] -Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#76, customer_id#97, year_total#98] +(72) Project [codegen id : 24] +Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102] +Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#78, customer_id#101, year_total#102] -(71) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +(73) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] +(74) ColumnarToRow [codegen id : 18] +Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -(73) Filter [codegen id : 18] -Input [8]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106] -Condition : (isnotnull(c_customer_sk#99) AND isnotnull(c_customer_id#100)) +(75) Filter [codegen id : 18] +Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] +Condition : (isnotnull(c_customer_sk#103) AND isnotnull(c_customer_id#104)) -(74) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +(76) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#111, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#116), dynamicpruningexpression(ws_sold_date_sk#116 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +(77) ColumnarToRow [codegen id : 16] +Input [6]: [ws_bill_customer_sk#111, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116] -(76) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] -Condition : isnotnull(ws_bill_customer_sk#107) +(78) Filter [codegen id : 16] +Input [6]: [ws_bill_customer_sk#111, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116] +Condition : isnotnull(ws_bill_customer_sk#111) -(77) BroadcastExchange -Input [6]: [ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +(79) BroadcastExchange +Input [6]: [ws_bill_customer_sk#111, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#99] -Right keys [1]: [ws_bill_customer_sk#107] +(80) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_customer_sk#103] +Right keys [1]: [ws_bill_customer_sk#111] Join type: Inner Join condition: None -(79) Project [codegen id : 18] -Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] -Input [14]: [c_customer_sk#99, c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_bill_customer_sk#107, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112] +(81) Project [codegen id : 18] +Output [12]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116] +Input [14]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, ws_bill_customer_sk#111, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116] -(80) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#113, d_year#114] +(82) ReusedExchange [Reuses operator id: 116] +Output [2]: [d_date_sk#117, d_year#118] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#112] -Right keys [1]: [d_date_sk#113] +(83) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#116] +Right keys [1]: [d_date_sk#117] Join type: Inner Join condition: None -(82) Project [codegen id : 18] -Output [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#114] -Input [14]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, ws_sold_date_sk#112, d_date_sk#113, d_year#114] - -(83) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#114] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114] -Functions [1]: [partial_sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))] -Aggregate Attributes [2]: [sum#115, isEmpty#116] -Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114, sum#117, isEmpty#118] - -(84) Exchange -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114, sum#117, isEmpty#118] -Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(85) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114, sum#117, isEmpty#118] -Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114] -Functions [1]: [sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))#119] -Results [2]: [c_customer_id#100 AS customer_id#120, sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))#119 AS year_total#121] - -(86) Filter [codegen id : 19] -Input [2]: [customer_id#120, year_total#121] -Condition : (isnotnull(year_total#121) AND (year_total#121 > 0.000000)) - -(87) BroadcastExchange -Input [2]: [customer_id#120, year_total#121] +(84) Project [codegen id : 18] +Output [12]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, d_year#118] +Input [14]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, ws_sold_date_sk#116, d_date_sk#117, d_year#118] + +(85) HashAggregate [codegen id : 18] +Input [12]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, ws_ext_discount_amt#112, ws_ext_sales_price#113, ws_ext_wholesale_cost#114, ws_ext_list_price#115, d_year#118] +Keys [8]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, d_year#118] +Functions [1]: [partial_sum(((((ws_ext_list_price#115 - ws_ext_wholesale_cost#114) - ws_ext_discount_amt#112) + ws_ext_sales_price#113) / 2))] +Aggregate Attributes [2]: [sum#119, isEmpty#120] +Results [10]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, d_year#118, sum#121, isEmpty#122] + +(86) Exchange +Input [10]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, d_year#118, sum#121, isEmpty#122] +Arguments: hashpartitioning(c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, d_year#118, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(87) HashAggregate [codegen id : 19] +Input [10]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, d_year#118, sum#121, isEmpty#122] +Keys [8]: [c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110, d_year#118] +Functions [1]: [sum(((((ws_ext_list_price#115 - ws_ext_wholesale_cost#114) - ws_ext_discount_amt#112) + ws_ext_sales_price#113) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#115 - ws_ext_wholesale_cost#114) - ws_ext_discount_amt#112) + ws_ext_sales_price#113) / 2))#123] +Results [2]: [c_customer_id#104 AS customer_id#124, sum(((((ws_ext_list_price#115 - ws_ext_wholesale_cost#114) - ws_ext_discount_amt#112) + ws_ext_sales_price#113) / 2))#123 AS year_total#125] + +(88) Filter [codegen id : 19] +Input [2]: [customer_id#124, year_total#125] +Condition : (isnotnull(year_total#125) AND (year_total#125 > 0.000000)) + +(89) Project [codegen id : 19] +Output [2]: [customer_id#124 AS customer_id#126, year_total#125 AS year_total#127] +Input [2]: [customer_id#124, year_total#125] + +(90) BroadcastExchange +Input [2]: [customer_id#126, year_total#127] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] -(88) BroadcastHashJoin [codegen id : 24] +(91) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#120] +Right keys [1]: [customer_id#126] Join type: Inner Join condition: None -(89) Project [codegen id : 24] -Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98, year_total#121] -Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98, customer_id#120, year_total#121] +(92) Project [codegen id : 24] +Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102, year_total#127] +Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102, customer_id#126, year_total#127] -(90) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] +(93) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#128, c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] +(94) ColumnarToRow [codegen id : 22] +Input [8]: [c_customer_sk#128, c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135] -(92) Filter [codegen id : 22] -Input [8]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129] -Condition : (isnotnull(c_customer_sk#122) AND isnotnull(c_customer_id#123)) +(95) Filter [codegen id : 22] +Input [8]: [c_customer_sk#128, c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135] +Condition : (isnotnull(c_customer_sk#128) AND isnotnull(c_customer_id#129)) -(93) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +(96) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#136, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#141), dynamicpruningexpression(ws_sold_date_sk#141 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +(97) ColumnarToRow [codegen id : 20] +Input [6]: [ws_bill_customer_sk#136, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141] -(95) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] -Condition : isnotnull(ws_bill_customer_sk#130) +(98) Filter [codegen id : 20] +Input [6]: [ws_bill_customer_sk#136, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141] +Condition : isnotnull(ws_bill_customer_sk#136) -(96) BroadcastExchange -Input [6]: [ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +(99) BroadcastExchange +Input [6]: [ws_bill_customer_sk#136, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -(97) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#122] -Right keys [1]: [ws_bill_customer_sk#130] +(100) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [c_customer_sk#128] +Right keys [1]: [ws_bill_customer_sk#136] Join type: Inner Join condition: None -(98) Project [codegen id : 22] -Output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] -Input [14]: [c_customer_sk#122, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_bill_customer_sk#130, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135] +(101) Project [codegen id : 22] +Output [12]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141] +Input [14]: [c_customer_sk#128, c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, ws_bill_customer_sk#136, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141] -(99) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#136, d_year#137] +(102) ReusedExchange [Reuses operator id: 120] +Output [2]: [d_date_sk#142, d_year#143] -(100) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#135] -Right keys [1]: [d_date_sk#136] +(103) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#141] +Right keys [1]: [d_date_sk#142] Join type: Inner Join condition: None -(101) Project [codegen id : 22] -Output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#137] -Input [14]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, ws_sold_date_sk#135, d_date_sk#136, d_year#137] - -(102) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#137] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137] -Functions [1]: [partial_sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))] -Aggregate Attributes [2]: [sum#138, isEmpty#139] -Results [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137, sum#140, isEmpty#141] - -(103) Exchange -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137, sum#140, isEmpty#141] -Arguments: hashpartitioning(c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(104) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137, sum#140, isEmpty#141] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137] -Functions [1]: [sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))#119] -Results [2]: [c_customer_id#123 AS customer_id#142, sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))#119 AS year_total#143] - -(105) BroadcastExchange -Input [2]: [customer_id#142, year_total#143] +(104) Project [codegen id : 22] +Output [12]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, d_year#143] +Input [14]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, ws_sold_date_sk#141, d_date_sk#142, d_year#143] + +(105) HashAggregate [codegen id : 22] +Input [12]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, ws_ext_discount_amt#137, ws_ext_sales_price#138, ws_ext_wholesale_cost#139, ws_ext_list_price#140, d_year#143] +Keys [8]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, d_year#143] +Functions [1]: [partial_sum(((((ws_ext_list_price#140 - ws_ext_wholesale_cost#139) - ws_ext_discount_amt#137) + ws_ext_sales_price#138) / 2))] +Aggregate Attributes [2]: [sum#144, isEmpty#145] +Results [10]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, d_year#143, sum#146, isEmpty#147] + +(106) Exchange +Input [10]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, d_year#143, sum#146, isEmpty#147] +Arguments: hashpartitioning(c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, d_year#143, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(107) HashAggregate [codegen id : 23] +Input [10]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, d_year#143, sum#146, isEmpty#147] +Keys [8]: [c_customer_id#129, c_first_name#130, c_last_name#131, c_preferred_cust_flag#132, c_birth_country#133, c_login#134, c_email_address#135, d_year#143] +Functions [1]: [sum(((((ws_ext_list_price#140 - ws_ext_wholesale_cost#139) - ws_ext_discount_amt#137) + ws_ext_sales_price#138) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#140 - ws_ext_wholesale_cost#139) - ws_ext_discount_amt#137) + ws_ext_sales_price#138) / 2))#123] +Results [2]: [c_customer_id#129 AS customer_id#148, sum(((((ws_ext_list_price#140 - ws_ext_wholesale_cost#139) - ws_ext_discount_amt#137) + ws_ext_sales_price#138) / 2))#123 AS year_total#149] + +(108) Project [codegen id : 23] +Output [2]: [customer_id#148 AS customer_id#150, year_total#149 AS year_total#151] +Input [2]: [customer_id#148, year_total#149] + +(109) BroadcastExchange +Input [2]: [customer_id#150, year_total#151] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] -(106) BroadcastHashJoin [codegen id : 24] +(110) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#142] +Right keys [1]: [customer_id#150] Join type: Inner -Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#121 > 0.000000) THEN (year_total#143 / year_total#121) END) +Join condition: (CASE WHEN (year_total#78 > 0.000000) THEN (year_total#102 / year_total#78) END > CASE WHEN (year_total#127 > 0.000000) THEN (year_total#151 / year_total#127) END) -(107) Project [codegen id : 24] +(111) Project [codegen id : 24] Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] -Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98, year_total#121, customer_id#142, year_total#143] +Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#78, year_total#102, year_total#127, customer_id#150, year_total#151] -(108) TakeOrderedAndProject +(112) TakeOrderedAndProject Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (112) -+- * Filter (111) - +- * ColumnarToRow (110) - +- Scan parquet spark_catalog.default.date_dim (109) +BroadcastExchange (116) ++- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet spark_catalog.default.date_dim (113) -(109) Scan parquet spark_catalog.default.date_dim +(113) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(110) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(111) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(112) BroadcastExchange +(116) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (116) -+- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (120) ++- * Filter (119) + +- * ColumnarToRow (118) + +- Scan parquet spark_catalog.default.date_dim (117) -(113) Scan parquet spark_catalog.default.date_dim +(117) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 1] +(118) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] -(115) Filter [codegen id : 1] +(119) Filter [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(116) BroadcastExchange +(120) BroadcastExchange Input [2]: [d_date_sk#40, d_year#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#90 IN dynamicpruning#39 +Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#92 IN dynamicpruning#39 -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 76 Hosting Expression = ws_sold_date_sk#116 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#39 +Subquery:6 Hosting operator id = 96 Hosting Expression = ws_sold_date_sk#141 IN dynamicpruning#39 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 5652036650504..dc36fc1777336 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -74,106 +74,110 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Filter [cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (19) + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + WholeStageCodegen (18) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [cs_bill_customer_sk] + BroadcastExchange #16 + WholeStageCodegen (16) + Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #17 WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 + WholeStageCodegen (22) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #19 + WholeStageCodegen (20) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 14ec7db1a8b8e..6bc3c6747b31c 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 (78) -+- * HashAggregate (77) - +- Exchange (76) - +- * HashAggregate (75) - +- * Expand (74) - +- Union (73) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- * Expand (73) + +- Union (72) :- * HashAggregate (21) : +- Exchange (20) : +- * HashAggregate (19) @@ -47,36 +47,35 @@ TakeOrderedAndProject (78) : : +- * ColumnarToRow (32) : : +- Scan parquet spark_catalog.default.catalog_page (31) : +- ReusedExchange (37) - +- * HashAggregate (72) - +- Exchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- Union (60) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- Union (59) : : :- * Project (46) : : : +- * Filter (45) : : : +- * ColumnarToRow (44) : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : +- * 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) + : : +- * 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) (1) Scan parquet spark_catalog.default.store_sales @@ -147,7 +146,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: 83] +(16) ReusedExchange [Reuses operator id: 82] Output [1]: [d_date_sk#24] (17) BroadcastHashJoin [codegen id : 5] @@ -246,7 +245,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: 83] +(37) ReusedExchange [Reuses operator id: 82] Output [1]: [d_date_sk#64] (38) BroadcastHashJoin [codegen id : 11] @@ -301,178 +300,173 @@ 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) 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 +(49) 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] -(51) Sort [codegen id : 15] +(50) 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 -(52) Scan parquet spark_catalog.default.web_sales +(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 -(53) ColumnarToRow [codegen id : 16] +(52) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(54) Filter [codegen id : 16] +(53) 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)) -(55) Project [codegen id : 16] +(54) 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] -(56) Exchange +(55) 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] -(57) Sort [codegen id : 17] +(56) 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 -(58) SortMergeJoin [codegen id : 18] +(57) 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 -(59) Project [codegen id : 18] +(58) 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] -(60) Union +(59) Union -(61) Scan parquet spark_catalog.default.web_site +(60) 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 -(62) ColumnarToRow [codegen id : 19] +(61) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] -(63) Filter [codegen id : 19] +(62) Filter [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] Condition : isnotnull(web_site_sk#107) -(64) BroadcastExchange +(63) 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] -(65) BroadcastHashJoin [codegen id : 21] +(64) 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 -(66) Project [codegen id : 21] +(65) 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] -(67) ReusedExchange [Reuses operator id: 83] +(66) ReusedExchange [Reuses operator id: 82] Output [1]: [d_date_sk#109] -(68) BroadcastHashJoin [codegen id : 21] +(67) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(69) Project [codegen id : 21] +(68) 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] -(70) HashAggregate [codegen id : 21] +(69) 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] -(71) Exchange +(70) 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] -(72) HashAggregate [codegen id : 22] +(71) 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] -(73) Union +(72) Union -(74) Expand [codegen id : 23] +(73) 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] -(75) HashAggregate [codegen id : 23] +(74) 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] -(76) Exchange +(75) 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] -(77) HashAggregate [codegen id : 24] +(76) 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] -(78) TakeOrderedAndProject +(77) 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 (83) -+- * Project (82) - +- * Filter (81) - +- * ColumnarToRow (80) - +- Scan parquet spark_catalog.default.date_dim (79) +BroadcastExchange (82) ++- * Project (81) + +- * Filter (80) + +- * ColumnarToRow (79) + +- Scan parquet spark_catalog.default.date_dim (78) -(79) Scan parquet spark_catalog.default.date_dim +(78) 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 -(80) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_date#148] -(81) Filter [codegen id : 1] +(80) 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)) -(82) Project [codegen id : 1] +(81) Project [codegen id : 1] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#148] -(83) BroadcastExchange +(82) 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 8395c291e8365..2547db4f5e5fb 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,11 +113,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 WholeStageCodegen (14) - 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 + 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 2d2c4c19891a2..313959456c809 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 (75) -+- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Expand (71) - +- Union (70) +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- * Expand (70) + +- Union (69) :- * HashAggregate (21) : +- Exchange (20) : +- * HashAggregate (19) @@ -47,33 +47,32 @@ TakeOrderedAndProject (75) : +- * Filter (36) : +- * ColumnarToRow (35) : +- Scan parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (60) - : +- * BroadcastHashJoin Inner BuildRight (59) - : :- Union (57) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- Union (56) : : :- * Project (46) : : : +- * Filter (45) : : : +- * ColumnarToRow (44) : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : +- * 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) + : : +- * 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) (1) Scan parquet spark_catalog.default.store_sales @@ -116,7 +115,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: 80] +(10) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#22] (11) BroadcastHashJoin [codegen id : 5] @@ -215,7 +214,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: 80] +(31) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#62] (32) BroadcastHashJoin [codegen id : 11] @@ -298,166 +297,161 @@ 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) Filter [codegen id : 14] +(49) BroadcastExchange 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)) +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(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 +(50) 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 +(51) ColumnarToRow Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(53) Filter +(52) 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)) -(54) Project +(53) 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] -(55) BroadcastHashJoin [codegen id : 15] +(54) 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 -(56) Project [codegen id : 15] +(55) 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] -(57) Union +(56) Union -(58) ReusedExchange [Reuses operator id: 80] +(57) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#107] -(59) BroadcastHashJoin [codegen id : 18] +(58) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(60) Project [codegen id : 18] +(59) 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] -(61) Scan parquet spark_catalog.default.web_site +(60) 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 -(62) ColumnarToRow [codegen id : 17] +(61) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] -(63) Filter [codegen id : 17] +(62) Filter [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] Condition : isnotnull(web_site_sk#108) -(64) BroadcastExchange +(63) 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] -(65) BroadcastHashJoin [codegen id : 18] +(64) 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 -(66) Project [codegen id : 18] +(65) 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] -(67) HashAggregate [codegen id : 18] +(66) 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] -(68) Exchange +(67) 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] -(69) HashAggregate [codegen id : 19] +(68) 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] -(70) Union +(69) Union -(71) Expand [codegen id : 20] +(70) 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] -(72) HashAggregate [codegen id : 20] +(71) 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] -(73) Exchange +(72) 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] -(74) HashAggregate [codegen id : 21] +(73) 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] -(75) TakeOrderedAndProject +(74) 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 (80) -+- * Project (79) - +- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (79) ++- * Project (78) + +- * Filter (77) + +- * ColumnarToRow (76) + +- Scan parquet spark_catalog.default.date_dim (75) -(76) Scan parquet spark_catalog.default.date_dim +(75) 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 -(77) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_date#148] -(78) Filter [codegen id : 1] +(77) 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)) -(79) Project [codegen id : 1] +(78) Project [codegen id : 1] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#148] -(80) BroadcastExchange +(79) 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 476811b554039..c6cd47e3f48ef 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,11 +110,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - 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 + 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/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index a7c9ffde5491b..d0cbf7ba82fe5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_id), IsNotNull(i_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,7 +136,7 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) +Condition : (isnotnull(i_item_id#10) AND isnotnull(i_item_sk#9)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_color#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 9daca6985443b..e28f119c554fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index a7c9ffde5491b..d0cbf7ba82fe5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_id), IsNotNull(i_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,7 +136,7 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) +Condition : (isnotnull(i_item_id#10) AND isnotnull(i_item_sk#9)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_color#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 9daca6985443b..e28f119c554fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 08d5d361f1aa9..74056a0043104 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -306,7 +306,7 @@ BroadcastExchange (60) Output [2]: [d_date_sk#5, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] @@ -314,7 +314,7 @@ Input [2]: [d_date_sk#5, d_date#39] (52) Filter [codegen id : 2] Input [2]: [d_date_sk#5, d_date#39] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#39)) +Condition : (isnotnull(d_date#39) AND isnotnull(d_date_sk#5)) (53) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 0947e6de1569e..952b03f746da8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index fd40efb9bb5c6..62c98f88ef856 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -306,7 +306,7 @@ BroadcastExchange (60) Output [2]: [d_date_sk#7, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 2] @@ -314,7 +314,7 @@ Input [2]: [d_date_sk#7, d_date#39] (52) Filter [codegen id : 2] Input [2]: [d_date_sk#7, d_date#39] -Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#39)) +Condition : (isnotnull(d_date#39) AND isnotnull(d_date_sk#7)) (53) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 95cf2d7f56c7a..9c2e3ae8e5786 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 51d4a65fd3f8d..0399be0216ef7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_id), IsNotNull(i_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,7 +136,7 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) +Condition : (isnotnull(i_item_id#10) AND isnotnull(i_item_sk#9)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_category#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 65e5199b9de8e..6f254e406e45d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 51d4a65fd3f8d..0399be0216ef7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -128,7 +128,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] Output [2]: [i_item_sk#9, i_item_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_id), IsNotNull(i_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -136,7 +136,7 @@ Input [2]: [i_item_sk#9, i_item_id#10] (16) Filter [codegen id : 4] Input [2]: [i_item_sk#9, i_item_id#10] -Condition : (isnotnull(i_item_sk#9) AND isnotnull(i_item_id#10)) +Condition : (isnotnull(i_item_id#10) AND isnotnull(i_item_sk#9)) (17) Scan parquet spark_catalog.default.item Output [2]: [i_item_id#11, i_category#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 65e5199b9de8e..6f254e406e45d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt index e49c1bdfec192..4350fa137ca6b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.web_site (11) - : +- BroadcastExchange (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.ship_mode (17) - +- BroadcastExchange (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet spark_catalog.default.warehouse (23) +TakeOrderedAndProject (33) ++- * HashAggregate (32) + +- Exchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet spark_catalog.default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet spark_catalog.default.web_site (11) + : +- BroadcastExchange (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.ship_mode (17) + +- BroadcastExchange (26) + +- * Filter (25) + +- * ColumnarToRow (24) + +- Scan parquet spark_catalog.default.warehouse (23) (1) Scan parquet spark_catalog.default.web_sales @@ -160,28 +161,32 @@ Join type: Inner Join condition: None (28) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#11, web_name#9, substr(w_warehouse_name#13, 1, 20) AS _groupingexpression#14] +Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#13, sm_type#11, web_name#9] Input [7]: [ws_ship_date_sk#1, ws_warehouse_sk#4, ws_sold_date_sk#5, web_name#9, sm_type#11, w_warehouse_sk#12, w_warehouse_name#13] -(29) HashAggregate [codegen id : 5] +(29) Project [codegen id : 5] +Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#11, web_name#9, substr(w_warehouse_name#13, 1, 20) AS _groupingexpression#14] +Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#13, sm_type#11, web_name#9] + +(30) HashAggregate [codegen id : 5] Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#11, web_name#9, _groupingexpression#14] Keys [3]: [_groupingexpression#14, sm_type#11, web_name#9] Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#11, web_name#9, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange +(31) Exchange Input [8]: [_groupingexpression#14, sm_type#11, web_name#9, sum#20, sum#21, sum#22, sum#23, sum#24] Arguments: hashpartitioning(_groupingexpression#14, sm_type#11, web_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 6] Input [8]: [_groupingexpression#14, sm_type#11, web_name#9, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#11, web_name#9] Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#11, web_name#9, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(32) TakeOrderedAndProject +(33) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#11, web_name#9, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#11 ASC NULLS FIRST, web_name#9 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#11, web_name#9, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt index 038fed63f9dda..b7f9068ebb635 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt @@ -6,43 +6,44 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (5) HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Project [ws_ship_date_sk,ws_warehouse_sk,ws_sold_date_sk,web_name,sm_type] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,web_name] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [web_site_sk] + Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Project [ws_ship_date_sk,ws_warehouse_sk,ws_sold_date_sk,web_name,sm_type] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,web_name] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.web_site [web_site_sk,web_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [sm_ship_mode_sk] - ColumnarToRow + Scan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] InputAdapter - Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [w_warehouse_sk] - ColumnarToRow + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_site [web_site_sk,web_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Filter [sm_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [w_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index f2e978d074582..3aa4afe70e4e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -55,7 +55,7 @@ TakeOrderedAndProject (50) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -63,7 +63,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) +Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) AND isnotnull(c_current_cdemo_sk#2)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index a066f9555cc33..8f61c6c78a7ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 55556, 899992, 0, 0),bloomFilter,buf] Exchange #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 7a12ef2307550..b3a6f06ceaf4e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -50,7 +50,7 @@ TakeOrderedAndProject (45) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -58,7 +58,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : ((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index f67b6440a63e5..7383b11f73c81 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 4cf423817ee2d..628b2c239e78b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -77,7 +77,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 8] @@ -85,7 +85,7 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] (9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) +Condition : (isnotnull(s_state#8) AND isnotnull(s_store_sk#6)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index 6821b542e8dce..a263a2daa1eee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 220fcb317b9bf..91eed48cb2734 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -77,7 +77,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 8] @@ -85,7 +85,7 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] (9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) +Condition : (isnotnull(s_state#8) AND isnotnull(s_store_sk#6)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 288863f322b9f..cc4fb41854a10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt index 820b45f24d780..66e775db18346 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * SortMergeJoin Inner (77) - :- * Project (59) - : +- * SortMergeJoin Inner (58) +TakeOrderedAndProject (81) ++- * Project (80) + +- * SortMergeJoin Inner (79) + :- * Project (60) + : +- * SortMergeJoin Inner (59) : :- * SortMergeJoin Inner (39) : : :- * Sort (21) : : : +- Exchange (20) @@ -43,41 +43,43 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (25) : : +- * Sort (31) : : +- ReusedExchange (30) - : +- * Sort (57) - : +- Exchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- Exchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * SortMergeJoin Inner (50) - : :- * Sort (47) - : : +- Exchange (46) - : : +- * Project (45) - : : +- * BroadcastHashJoin Inner BuildRight (44) - : : :- * Filter (42) - : : : +- * ColumnarToRow (41) - : : : +- Scan parquet spark_catalog.default.web_sales (40) - : : +- ReusedExchange (43) - : +- * Sort (49) - : +- ReusedExchange (48) - +- * Sort (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * SortMergeJoin Inner (70) - :- * Sort (67) - : +- Exchange (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Filter (62) - : : +- * ColumnarToRow (61) - : : +- Scan parquet spark_catalog.default.web_sales (60) - : +- ReusedExchange (63) - +- * Sort (69) - +- ReusedExchange (68) + : +- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- Exchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * SortMergeJoin Inner (50) + : :- * Sort (47) + : : +- Exchange (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- * Filter (42) + : : : +- * ColumnarToRow (41) + : : : +- Scan parquet spark_catalog.default.web_sales (40) + : : +- ReusedExchange (43) + : +- * Sort (49) + : +- ReusedExchange (48) + +- * Sort (78) + +- Exchange (77) + +- * Project (76) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * SortMergeJoin Inner (71) + :- * Sort (68) + : +- Exchange (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Filter (63) + : : +- * ColumnarToRow (62) + : : +- Scan parquet spark_catalog.default.web_sales (61) + : +- ReusedExchange (64) + +- * Sort (70) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.store_sales @@ -95,7 +97,7 @@ Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 83] +(4) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#5, d_year#6] (5) BroadcastHashJoin [codegen id : 2] @@ -193,7 +195,7 @@ Input [3]: [ss_customer_sk#16, ss_net_paid#17, ss_sold_date_sk#18] Input [3]: [ss_customer_sk#16, ss_net_paid#17, ss_sold_date_sk#18] Condition : isnotnull(ss_customer_sk#16) -(25) ReusedExchange [Reuses operator id: 87] +(25) ReusedExchange [Reuses operator id: 89] Output [2]: [d_date_sk#20, d_year#21] (26) BroadcastHashJoin [codegen id : 10] @@ -278,7 +280,7 @@ Input [3]: [ws_bill_customer_sk#32, ws_net_paid#33, ws_sold_date_sk#34] Input [3]: [ws_bill_customer_sk#32, ws_net_paid#33, ws_sold_date_sk#34] Condition : isnotnull(ws_bill_customer_sk#32) -(43) ReusedExchange [Reuses operator id: 83] +(43) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#35, d_year#36] (44) BroadcastHashJoin [codegen id : 19] @@ -338,171 +340,179 @@ Results [2]: [c_customer_id#38 AS customer_id#44, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#44, year_total#45] Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) -(56) Exchange +(56) Project [codegen id : 24] +Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] Input [2]: [customer_id#44, year_total#45] -Arguments: hashpartitioning(customer_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(57) Sort [codegen id : 25] -Input [2]: [customer_id#44, year_total#45] -Arguments: [customer_id#44 ASC NULLS FIRST], false, 0 +(57) Exchange +Input [2]: [customer_id#46, year_total#47] +Arguments: hashpartitioning(customer_id#46, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(58) Sort [codegen id : 25] +Input [2]: [customer_id#46, year_total#47] +Arguments: [customer_id#46 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 26] +(59) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#44] +Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(59) Project [codegen id : 26] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#44, year_total#45] +(60) Project [codegen id : 26] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -(60) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +(61) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +(62) ColumnarToRow [codegen id : 28] +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -(62) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] -Condition : isnotnull(ws_bill_customer_sk#46) +(63) Filter [codegen id : 28] +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Condition : isnotnull(ws_bill_customer_sk#48) -(63) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#49, d_year#50] +(64) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#51, d_year#52] -(64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +(65) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ws_sold_date_sk#50] +Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(65) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50] -Input [5]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48, d_date_sk#49, d_year#50] +(66) Project [codegen id : 28] +Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52] +Input [5]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50, d_date_sk#51, d_year#52] -(66) Exchange -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50] -Arguments: hashpartitioning(ws_bill_customer_sk#46, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(67) Exchange +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52] +Arguments: hashpartitioning(ws_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(67) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50] -Arguments: [ws_bill_customer_sk#46 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 29] +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52] +Arguments: [ws_bill_customer_sk#48 ASC NULLS FIRST], false, 0 -(68) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +(69) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] -(69) Sort [codegen id : 31] -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 31] +Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +Arguments: [c_customer_sk#53 ASC NULLS FIRST], false, 0 -(70) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#46] -Right keys [1]: [c_customer_sk#51] +(71) SortMergeJoin [codegen id : 32] +Left keys [1]: [ws_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#53] Join type: Inner Join condition: None -(71) Project [codegen id : 32] -Output [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, ws_net_paid#47, d_year#50] -Input [7]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50, c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] - -(72) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, ws_net_paid#47, d_year#50] -Keys [4]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#47))] -Aggregate Attributes [1]: [sum#55] -Results [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, sum#56] - -(73) Exchange -Input [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, sum#56] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(74) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, sum#56] -Keys [4]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_net_paid#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#47))#43] -Results [2]: [c_customer_id#52 AS customer_id#57, MakeDecimal(sum(UnscaledValue(ws_net_paid#47))#43,17,2) AS year_total#58] - -(75) Exchange -Input [2]: [customer_id#57, year_total#58] -Arguments: hashpartitioning(customer_id#57, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(76) Sort [codegen id : 34] -Input [2]: [customer_id#57, year_total#58] -Arguments: [customer_id#57 ASC NULLS FIRST], false, 0 - -(77) SortMergeJoin [codegen id : 35] +(72) Project [codegen id : 32] +Output [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, ws_net_paid#49, d_year#52] +Input [7]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52, c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] + +(73) HashAggregate [codegen id : 32] +Input [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, ws_net_paid#49, d_year#52] +Keys [4]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#49))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, sum#58] + +(74) Exchange +Input [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, sum#58] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(75) HashAggregate [codegen id : 33] +Input [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, sum#58] +Keys [4]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52] +Functions [1]: [sum(UnscaledValue(ws_net_paid#49))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#49))#43] +Results [2]: [c_customer_id#54 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#49))#43,17,2) AS year_total#60] + +(76) Project [codegen id : 33] +Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] +Input [2]: [customer_id#59, year_total#60] + +(77) Exchange +Input [2]: [customer_id#61, year_total#62] +Arguments: hashpartitioning(customer_id#61, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(78) Sort [codegen id : 34] +Input [2]: [customer_id#61, year_total#62] +Arguments: [customer_id#61 ASC NULLS FIRST], false, 0 + +(79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#57] +Right keys [1]: [customer_id#61] Join type: Inner -Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#47 > 0.00) THEN (year_total#62 / year_total#47) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(78) Project [codegen id : 35] +(80) Project [codegen id : 35] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45, customer_id#57, year_total#58] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#61, year_total#62] -(79) TakeOrderedAndProject +(81) TakeOrderedAndProject Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (85) ++- * Filter (84) + +- * ColumnarToRow (83) + +- Scan parquet spark_catalog.default.date_dim (82) -(80) Scan parquet spark_catalog.default.date_dim +(82) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_year#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_year#6] -(82) Filter [codegen id : 1] +(84) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_year#6] Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) -(83) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (87) -+- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (89) ++- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet spark_catalog.default.date_dim (86) -(84) Scan parquet spark_catalog.default.date_dim +(86) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(86) Filter [codegen id : 1] +(88) Filter [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] Condition : (((isnotnull(d_year#21) AND (d_year#21 = 2002)) AND d_year#21 IN (2001,2002)) AND isnotnull(d_date_sk#20)) -(87) BroadcastExchange +(89) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#34 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#19 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt index dca0cb8c44ab7..5bfda7e702a44 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt @@ -99,64 +99,66 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] InputAdapter Exchange [customer_id] #10 WholeStageCodegen (24) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (20) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (19) - Project [ws_bill_customer_sk,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #4 - InputAdapter - WholeStageCodegen (22) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + WholeStageCodegen (23) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - WholeStageCodegen (32) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (28) - Project [ws_bill_customer_sk,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #9 - InputAdapter - WholeStageCodegen (31) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + WholeStageCodegen (32) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (28) + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #9 + InputAdapter + WholeStageCodegen (31) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index d5f9b03d835af..ef23cd5f40912 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) +TakeOrderedAndProject (73) ++- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * BroadcastHashJoin Inner BuildRight (33) : : :- * Filter (16) : : : +- * HashAggregate (15) @@ -37,39 +37,41 @@ TakeOrderedAndProject (71) : : : +- * ColumnarToRow (21) : : : +- Scan parquet spark_catalog.default.store_sales (20) : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * Filter (39) - : : +- * ColumnarToRow (38) - : : +- Scan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) - +- BroadcastExchange (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * Filter (58) - : +- * ColumnarToRow (57) - : +- Scan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (70) + +- * Project (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (56) + : : +- * ColumnarToRow (55) + : : +- Scan parquet spark_catalog.default.customer (54) + : +- BroadcastExchange (60) + : +- * Filter (59) + : +- * ColumnarToRow (58) + : +- Scan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (63) (1) Scan parquet spark_catalog.default.customer @@ -115,7 +117,7 @@ Join condition: None Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(10) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#9, d_year#10] (11) BroadcastHashJoin [codegen id : 3] @@ -193,7 +195,7 @@ Join condition: None Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(26) ReusedExchange [Reuses operator id: 79] +(26) ReusedExchange [Reuses operator id: 81] Output [2]: [d_date_sk#24, d_year#25] (27) BroadcastHashJoin [codegen id : 6] @@ -277,7 +279,7 @@ Join condition: None Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(43) ReusedExchange [Reuses operator id: 75] +(43) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#39, d_year#40] (44) BroadcastHashJoin [codegen id : 10] @@ -312,166 +314,174 @@ Results [2]: [c_customer_id#33 AS customer_id#44, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#44, year_total#45] Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) -(50) BroadcastExchange +(50) Project [codegen id : 11] +Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] Input [2]: [customer_id#44, year_total#45] + +(51) BroadcastExchange +Input [2]: [customer_id#46, year_total#47] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#44] +Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#44, year_total#45] +(53) Project [codegen id : 16] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -(53) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] +(54) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] +(55) ColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -(55) Filter [codegen id : 14] -Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(c_customer_id#47)) +(56) Filter [codegen id : 14] +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Condition : (isnotnull(c_customer_sk#48) AND isnotnull(c_customer_id#49)) -(56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(57) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(58) ColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -(58) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_customer_sk#50) +(59) Filter [codegen id : 12] +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_customer_sk#52) -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(60) BroadcastExchange +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#46] -Right keys [1]: [ws_bill_customer_sk#50] +(61) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#48] +Right keys [1]: [ws_bill_customer_sk#52] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, ws_sold_date_sk#52] -Input [7]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(62) Project [codegen id : 14] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, ws_sold_date_sk#54] +Input [7]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#53, d_year#54] +(63) ReusedExchange [Reuses operator id: 81] +Output [2]: [d_date_sk#55, d_year#56] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#53] +(64) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, d_year#54] -Input [7]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54] - -(65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, d_year#54] -Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#51))] -Aggregate Attributes [1]: [sum#55] -Results [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56] - -(66) Exchange -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56] -Arguments: hashpartitioning(c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56] -Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_net_paid#51))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#51))#43] -Results [2]: [c_customer_id#47 AS customer_id#57, MakeDecimal(sum(UnscaledValue(ws_net_paid#51))#43,17,2) AS year_total#58] - -(68) BroadcastExchange -Input [2]: [customer_id#57, year_total#58] +(65) Project [codegen id : 14] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, d_year#56] +Input [7]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56] + +(66) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, d_year#56] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, sum#58] + +(67) Exchange +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(68) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, sum#58] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#53))#43] +Results [2]: [c_customer_id#49 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#53))#43,17,2) AS year_total#60] + +(69) Project [codegen id : 15] +Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] +Input [2]: [customer_id#59, year_total#60] + +(70) BroadcastExchange +Input [2]: [customer_id#61, year_total#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#57] +Right keys [1]: [customer_id#61] Join type: Inner -Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#47 > 0.00) THEN (year_total#62 / year_total#47) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(70) Project [codegen id : 16] +(72) Project [codegen id : 16] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45, customer_id#57, year_total#58] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#61, year_total#62] -(71) TakeOrderedAndProject +(73) TakeOrderedAndProject Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * Filter (76) + +- * ColumnarToRow (75) + +- Scan parquet spark_catalog.default.date_dim (74) -(72) Scan parquet spark_catalog.default.date_dim +(74) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(74) Filter [codegen id : 1] +(76) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(75) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (79) -+- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (81) ++- * Filter (80) + +- * ColumnarToRow (79) + +- Scan parquet spark_catalog.default.date_dim (78) -(76) Scan parquet spark_catalog.default.date_dim +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(77) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(78) Filter [codegen id : 1] +(80) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) -(79) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 6d762b4937f96..001bf887409de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -70,53 +70,55 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 1642f1b57c82d..a809fa840b70e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -1,48 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- Union (40) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet spark_catalog.default.date_dim (4) - : +- BroadcastExchange (13) - : +- * Filter (12) - : +- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.item (10) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildLeft (29) - : :- BroadcastExchange (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildLeft (23) - : : :- BroadcastExchange (19) - : : : +- * Filter (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.web_sales (16) - : : +- * Filter (22) - : : +- * ColumnarToRow (21) - : : +- Scan parquet spark_catalog.default.date_dim (20) - : +- * Filter (28) - : +- * ColumnarToRow (27) - : +- Scan parquet spark_catalog.default.item (26) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet spark_catalog.default.catalog_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- Union (43) + :- * Project (16) + : +- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet spark_catalog.default.date_dim (4) + : +- BroadcastExchange (13) + : +- * Filter (12) + : +- * ColumnarToRow (11) + : +- Scan parquet spark_catalog.default.item (10) + :- * Project (32) + : +- * Project (31) + : +- * BroadcastHashJoin Inner BuildLeft (30) + : :- BroadcastExchange (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildLeft (24) + : : :- BroadcastExchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet spark_catalog.default.web_sales (17) + : : +- * Filter (23) + : : +- * ColumnarToRow (22) + : : +- Scan parquet spark_catalog.default.date_dim (21) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet spark_catalog.default.item (27) + +- * Project (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Filter (35) + : : +- * ColumnarToRow (34) + : : +- Scan parquet spark_catalog.default.catalog_sales (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) (1) Scan parquet spark_catalog.default.store_sales @@ -113,10 +116,14 @@ Join type: Inner Join condition: None (15) Project [codegen id : 3] -Output [6]: [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#6, d_qoy#7, i_category#9, ss_ext_sales_price#3 AS ext_sales_price#12] +Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, i_category#9, d_year#6, d_qoy#7] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, d_year#6, d_qoy#7, i_item_sk#8, i_category#9] -(16) Scan parquet spark_catalog.default.web_sales +(16) Project [codegen id : 3] +Output [6]: [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#6, d_qoy#7, i_category#9, ss_ext_sales_price#3 AS ext_sales_price#12] +Input [5]: [ss_store_sk#2, ss_ext_sales_price#3, i_category#9, d_year#6, d_qoy#7] + +(17) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] @@ -124,70 +131,74 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#16)] PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(18) ColumnarToRow [codegen id : 4] Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] -(18) Filter [codegen id : 4] +(19) Filter [codegen id : 4] Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Condition : (isnull(ws_ship_customer_sk#14) AND isnotnull(ws_item_sk#13)) -(19) BroadcastExchange +(20) BroadcastExchange Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [plan_id=3] -(20) Scan parquet spark_catalog.default.date_dim +(21) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(21) ColumnarToRow +(22) ColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -(22) Filter +(23) Filter Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] Condition : isnotnull(d_date_sk#17) -(23) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(24) Project [codegen id : 5] +(25) Project [codegen id : 5] Output [5]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, d_year#18, d_qoy#19] Input [7]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, d_date_sk#17, d_year#18, d_qoy#19] -(25) BroadcastExchange +(26) BroadcastExchange Input [5]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, d_year#18, d_qoy#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(26) Scan parquet spark_catalog.default.item +(27) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#20, i_category#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(27) ColumnarToRow +(28) ColumnarToRow Input [2]: [i_item_sk#20, i_category#21] -(28) Filter +(29) Filter Input [2]: [i_item_sk#20, i_category#21] Condition : isnotnull(i_item_sk#20) -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#13] Right keys [1]: [i_item_sk#20] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#18, d_qoy#19, i_category#21, ws_ext_sales_price#15 AS ext_sales_price#24] +(31) Project [codegen id : 6] +Output [5]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, i_category#21, d_year#18, d_qoy#19] Input [7]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, d_year#18, d_qoy#19, i_item_sk#20, i_category#21] -(31) Scan parquet spark_catalog.default.catalog_sales +(32) Project [codegen id : 6] +Output [6]: [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#18, d_qoy#19, i_category#21, ws_ext_sales_price#15 AS ext_sales_price#24] +Input [5]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, i_category#21, d_year#18, d_qoy#19] + +(33) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] @@ -195,60 +206,64 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#28)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 9] +(34) ColumnarToRow [codegen id : 9] Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] -(33) Filter [codegen id : 9] +(35) Filter [codegen id : 9] Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Condition : (isnull(cs_ship_addr_sk#25) AND isnotnull(cs_item_sk#26)) -(34) ReusedExchange [Reuses operator id: 7] +(36) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#29, d_year#30, d_qoy#31] -(35) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#28] Right keys [1]: [d_date_sk#29] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(38) Project [codegen id : 9] Output [5]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, d_year#30, d_qoy#31] Input [7]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28, d_date_sk#29, d_year#30, d_qoy#31] -(37) ReusedExchange [Reuses operator id: 13] +(39) ReusedExchange [Reuses operator id: 13] Output [2]: [i_item_sk#32, i_category#33] -(38) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#26] Right keys [1]: [i_item_sk#32] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [6]: [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#30, d_qoy#31, i_category#33, cs_ext_sales_price#27 AS ext_sales_price#36] +(41) Project [codegen id : 9] +Output [5]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, i_category#33, d_year#30, d_qoy#31] Input [7]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, d_year#30, d_qoy#31, i_item_sk#32, i_category#33] -(40) Union +(42) Project [codegen id : 9] +Output [6]: [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#30, d_qoy#31, i_category#33, cs_ext_sales_price#27 AS ext_sales_price#36] +Input [5]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, i_category#33, d_year#30, d_qoy#31] + +(43) Union -(41) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 10] Input [6]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, ext_sales_price#12] Keys [5]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12))] Aggregate Attributes [2]: [count#37, sum#38] Results [7]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, count#39, sum#40] -(42) Exchange +(45) Exchange Input [7]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, count#39, sum#40] Arguments: hashpartitioning(channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 11] +(46) HashAggregate [codegen id : 11] Input [7]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, count#39, sum#40] Keys [5]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] Results [7]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] -(44) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [7]: [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, sales_cnt#43, sales_amt#44] Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [channel#10, col_name#11, d_year#6, d_qoy#7, i_category#9, sales_cnt#43, sales_amt#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt index 8ce1999b6e7a3..19f653c8432de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt @@ -9,60 +9,63 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ Union WholeStageCodegen (3) Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [i_item_sk] + Project [ss_store_sk,ss_ext_sales_price,i_category,d_year,d_qoy] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_category] + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_category] WholeStageCodegen (6) Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] - BroadcastHashJoin [ws_item_sk,i_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - Project [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [ws_ship_customer_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_category] + Project [ws_ship_customer_sk,ws_ext_sales_price,i_category,d_year,d_qoy] + BroadcastHashJoin [ws_item_sk,i_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + Project [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [ws_ship_customer_sk,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_category] WholeStageCodegen (9) Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #2 - InputAdapter - ReusedExchange [i_item_sk,i_category] #3 + Project [cs_ship_addr_sk,cs_ext_sales_price,i_category,d_year,d_qoy] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_ship_addr_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #2 + InputAdapter + ReusedExchange [i_item_sk,i_category] #3 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 afd1b852e7984..d443723b06386 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,7 +118,6 @@ 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] @@ -126,7 +125,7 @@ Input [1]: [ca_zip#9] (17) Filter [codegen id : 11] Input [1]: [ca_zip#9] -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))) +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))) (18) Project [codegen id : 11] Output [1]: [substr(ca_zip#9, 1, 5) AS ca_zip#10] 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 c235c1651e91f..24c3a657ddd20 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,7 +104,6 @@ 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] @@ -112,7 +111,7 @@ Input [1]: [ca_zip#9] (15) Filter [codegen id : 6] Input [1]: [ca_zip#9] -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))) +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))) (16) Project [codegen id : 6] Output [1]: [substr(ca_zip#9, 1, 5) AS ca_zip#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index cfe301a17a036..0248aebc1ac03 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -297,7 +297,7 @@ BroadcastExchange (63) Output [2]: [d_date_sk#5, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 3] @@ -305,7 +305,7 @@ Input [2]: [d_date_sk#5, d_date#39] (49) Filter [codegen id : 3] Input [2]: [d_date_sk#5, d_date#39] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#39)) +Condition : (isnotnull(d_date#39) AND isnotnull(d_date_sk#5)) (50) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index e141529e772f5..2b59271dba0d1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 11836de73a5d7..2e2904de01fe5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -297,7 +297,7 @@ BroadcastExchange (63) Output [2]: [d_date_sk#7, d_date#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 3] @@ -305,7 +305,7 @@ Input [2]: [d_date_sk#7, d_date#39] (49) Filter [codegen id : 3] Input [2]: [d_date_sk#7, d_date#39] -Condition : (isnotnull(d_date_sk#7) AND isnotnull(d_date#39)) +Condition : (isnotnull(d_date#39) AND isnotnull(d_date_sk#7)) (50) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#40, d_week_seq#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 05a4dd73a2783..d15b11a726954 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] 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 dc2f1572fa108..555befa355339 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 @@ -4,24 +4,24 @@ TakeOrderedAndProject (25) +- Exchange (23) +- * HashAggregate (22) +- * Project (21) - +- * SortMergeJoin Inner (20) - :- * Sort (13) - : +- Exchange (12) - : +- * Project (11) - : +- * BroadcastHashJoin Inner BuildRight (10) - : :- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_returns (1) - : +- BroadcastExchange (9) - : +- * Project (8) - : +- * Filter (7) - : +- * ColumnarToRow (6) - : +- Scan parquet spark_catalog.default.reason (5) - +- * Sort (19) - +- Exchange (18) - +- * Project (17) - +- * Filter (16) + +- * Project (20) + +- * SortMergeJoin Inner (19) + :- * Sort (13) + : +- Exchange (12) + : +- * Project (11) + : +- * BroadcastHashJoin Inner BuildRight (10) + : :- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet spark_catalog.default.store_returns (1) + : +- BroadcastExchange (9) + : +- * Project (8) + : +- * 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) @@ -88,37 +88,36 @@ 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) 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] +(16) 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] -(18) Exchange +(17) 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] -(19) Sort [codegen id : 5] +(18) 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 -(20) SortMergeJoin [codegen id : 6] +(19) 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] +Output [4]: [ss_customer_sk#9, ss_quantity#11, ss_sales_price#12, sr_return_quantity#4] +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) 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] +Input [4]: [ss_customer_sk#9, ss_quantity#11, ss_sales_price#12, sr_return_quantity#4] (22) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#9, act_sales#14] 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 92a735d0f3062..43c2d64624902 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 @@ -6,36 +6,36 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] WholeStageCodegen (6) HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - InputAdapter - WholeStageCodegen (3) - Sort [sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #2 - WholeStageCodegen (2) - Project [sr_item_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [sr_reason_sk,r_reason_sk] - Project [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - Filter [sr_item_sk,sr_ticket_number,sr_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [r_reason_sk] - Filter [r_reason_desc,r_reason_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] - InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk,ss_ticket_number] - InputAdapter - 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] - Filter [ss_item_sk,ss_ticket_number] + Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_return_quantity] + SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + InputAdapter + WholeStageCodegen (3) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #2 + WholeStageCodegen (2) + Project [sr_item_sk,sr_ticket_number,sr_return_quantity] + BroadcastHashJoin [sr_reason_sk,r_reason_sk] + Project [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + Filter [sr_item_sk,sr_ticket_number,sr_reason_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [r_reason_sk] + Filter [r_reason_desc,r_reason_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + InputAdapter + WholeStageCodegen (5) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + 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] 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 a6e8d395c0db5..172c03565451e 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,144 +1,138 @@ == Physical Plan == -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) +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) (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) 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] +(3) 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] -(5) Exchange +(4) 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] -(6) Sort [codegen id : 2] +(5) 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 -(7) Scan parquet spark_catalog.default.store_returns +(6) 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 -(8) ColumnarToRow [codegen id : 3] +(7) 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] -(9) Filter [codegen id : 3] +(8) 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)) -(10) Project [codegen id : 3] +(9) 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] -(11) Exchange +(10) 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] -(12) Sort [codegen id : 4] +(11) 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 -(13) SortMergeJoin [codegen id : 6] +(12) 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 -(14) Project [codegen id : 6] +(13) 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] -(15) Scan parquet spark_catalog.default.reason +(14) 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 -(16) ColumnarToRow [codegen id : 5] +(15) ColumnarToRow [codegen id : 5] Input [2]: [r_reason_sk#12, r_reason_desc#13] -(17) Filter [codegen id : 5] +(16) 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)) -(18) Project [codegen id : 5] +(17) Project [codegen id : 5] Output [1]: [r_reason_sk#12] Input [2]: [r_reason_sk#12, r_reason_desc#13] -(19) BroadcastExchange +(18) BroadcastExchange Input [1]: [r_reason_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#8] Right keys [1]: [r_reason_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 6] +(20) 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] -(22) HashAggregate [codegen id : 6] +(21) 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] -(23) Exchange +(22) Exchange Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(24) HashAggregate [codegen id : 7] +(23) 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] -(25) TakeOrderedAndProject +(24) 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 ad84f69a8836f..350956593ae88 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,10 +16,9 @@ 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] - 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] + 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-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index d3d332be3db5b..dfd5fd8069e4d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -52,7 +52,7 @@ Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -60,7 +60,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : (((((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(ws_ship_date_sk#1, 42))) +Condition : (((((((isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) AND isnotnull(ws_ship_date_sk#1)) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND might_contain(Subquery scalar-subquery#9, [id=#10], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#11, [id=#12], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#13, [id=#14], xxhash64(ws_ship_date_sk#1, 42))) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 5bed7df19ab42..f90d4b738149b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] + Filter [ws_order_number,ws_warehouse_sk,ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 17961, 333176, 0, 0),bloomFilter,buf] Exchange #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 648e67b7c0b34..2650168c66d91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -52,7 +52,7 @@ Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -60,7 +60,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) +Condition : ((((isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) AND isnotnull(ws_ship_date_sk#1)) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 9f5f02383e358..9cb3f0969d656 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -24,7 +24,7 @@ WholeStageCodegen (12) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] + Filter [ws_order_number,ws_warehouse_sk,ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index f3ce4a1d9c9e4..ee25c97a7beb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -62,7 +62,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +70,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) AND might_contain(Subquery scalar-subquery#8, [id=#9], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#10, [id=#11], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#12, [id=#13], xxhash64(ws_ship_date_sk#1, 42))) +Condition : ((((((isnotnull(ws_order_number#4) AND isnotnull(ws_ship_date_sk#1)) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND might_contain(Subquery scalar-subquery#8, [id=#9], xxhash64(ws_ship_addr_sk#2, 42))) AND might_contain(Subquery scalar-subquery#10, [id=#11], xxhash64(ws_web_site_sk#3, 42))) AND might_contain(Subquery scalar-subquery#12, [id=#13], xxhash64(ws_ship_date_sk#1, 42))) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index f55c841465bfe..4eb8a27ac9df2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (21) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] + Filter [ws_order_number,ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 17961, 333176, 0, 0),bloomFilter,buf] Exchange #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 513f72d9b86f7..3d3fa1256a73a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -63,7 +63,7 @@ Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +71,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) +Condition : (((isnotnull(ws_order_number#4) AND isnotnull(ws_ship_date_sk#1)) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 124708bbfbc21..297603d7b8a47 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (21) Exchange [ws_order_number] #2 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] + Filter [ws_order_number,ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt index 7d4029a13bb88..96ea19a61494d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.ship_mode (11) - : +- BroadcastExchange (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.call_center (17) - +- BroadcastExchange (26) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet spark_catalog.default.warehouse (23) +TakeOrderedAndProject (33) ++- * HashAggregate (32) + +- Exchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet spark_catalog.default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet spark_catalog.default.ship_mode (11) + : +- BroadcastExchange (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet spark_catalog.default.call_center (17) + +- BroadcastExchange (26) + +- * Filter (25) + +- * ColumnarToRow (24) + +- Scan parquet spark_catalog.default.warehouse (23) (1) Scan parquet spark_catalog.default.catalog_sales @@ -160,28 +161,32 @@ Join type: Inner Join condition: None (28) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#13, 1, 20) AS _groupingexpression#14] +Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#13, sm_type#9, cc_name#11] Input [7]: [cs_ship_date_sk#1, cs_warehouse_sk#4, cs_sold_date_sk#5, sm_type#9, cc_name#11, w_warehouse_sk#12, w_warehouse_name#13] -(29) HashAggregate [codegen id : 5] +(29) Project [codegen id : 5] +Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#13, 1, 20) AS _groupingexpression#14] +Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#13, sm_type#9, cc_name#11] + +(30) HashAggregate [codegen id : 5] Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange +(31) Exchange Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 6] Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(32) TakeOrderedAndProject +(33) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt index c0b405ce664f4..1d65756175bd4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt @@ -6,43 +6,44 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (5) HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_warehouse_sk,cs_sold_date_sk,sm_type,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_warehouse_sk,cs_sold_date_sk,sm_type] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [sm_ship_mode_sk] + Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_warehouse_sk,cs_sold_date_sk,sm_type,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_warehouse_sk,cs_sold_date_sk,sm_type] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [cc_call_center_sk] - ColumnarToRow + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] InputAdapter - Scan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [w_warehouse_sk] - ColumnarToRow + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [sm_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Filter [cc_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [w_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index b85e98570179d..d94b852d6b438 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -53,7 +53,7 @@ TakeOrderedAndProject (48) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) +Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) AND isnotnull(c_current_cdemo_sk#2)) AND might_contain(Subquery scalar-subquery#4, [id=#5], xxhash64(c_current_addr_sk#3, 42))) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 747b6ab4dd8cc..0a8fe12b89064 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] Subquery #1 ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 2555, 57765, 0, 0),bloomFilter,buf] Exchange #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 59c47a8b285a1..654f675059e79 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -49,7 +49,7 @@ TakeOrderedAndProject (44) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -57,7 +57,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : ((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 277b01735feff..fb8891293438d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index e37b867a26995..333d66dbf586d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * SortMergeJoin Inner (77) - :- * Project (59) - : +- * SortMergeJoin Inner (58) +TakeOrderedAndProject (81) ++- * Project (80) + +- * SortMergeJoin Inner (79) + :- * Project (60) + : +- * SortMergeJoin Inner (59) : :- * SortMergeJoin Inner (39) : : :- * Sort (21) : : : +- Exchange (20) @@ -43,41 +43,43 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (25) : : +- * Sort (31) : : +- ReusedExchange (30) - : +- * Sort (57) - : +- Exchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- Exchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * SortMergeJoin Inner (50) - : :- * Sort (47) - : : +- Exchange (46) - : : +- * Project (45) - : : +- * BroadcastHashJoin Inner BuildRight (44) - : : :- * Filter (42) - : : : +- * ColumnarToRow (41) - : : : +- Scan parquet spark_catalog.default.web_sales (40) - : : +- ReusedExchange (43) - : +- * Sort (49) - : +- ReusedExchange (48) - +- * Sort (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * SortMergeJoin Inner (70) - :- * Sort (67) - : +- Exchange (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Filter (62) - : : +- * ColumnarToRow (61) - : : +- Scan parquet spark_catalog.default.web_sales (60) - : +- ReusedExchange (63) - +- * Sort (69) - +- ReusedExchange (68) + : +- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- Exchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * SortMergeJoin Inner (50) + : :- * Sort (47) + : : +- Exchange (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- * Filter (42) + : : : +- * ColumnarToRow (41) + : : : +- Scan parquet spark_catalog.default.web_sales (40) + : : +- ReusedExchange (43) + : +- * Sort (49) + : +- ReusedExchange (48) + +- * Sort (78) + +- Exchange (77) + +- * Project (76) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * SortMergeJoin Inner (71) + :- * Sort (68) + : +- Exchange (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Filter (63) + : : +- * ColumnarToRow (62) + : : +- Scan parquet spark_catalog.default.web_sales (61) + : +- ReusedExchange (64) + +- * Sort (70) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.store_sales @@ -95,7 +97,7 @@ Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sol Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 83] +(4) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#6, d_year#7] (5) BroadcastHashJoin [codegen id : 2] @@ -193,7 +195,7 @@ Input [4]: [ss_customer_sk#21, ss_ext_discount_amt#22, ss_ext_list_price#23, ss_ Input [4]: [ss_customer_sk#21, ss_ext_discount_amt#22, ss_ext_list_price#23, ss_sold_date_sk#24] Condition : isnotnull(ss_customer_sk#21) -(25) ReusedExchange [Reuses operator id: 87] +(25) ReusedExchange [Reuses operator id: 89] Output [2]: [d_date_sk#26, d_year#27] (26) BroadcastHashJoin [codegen id : 10] @@ -278,7 +280,7 @@ Input [4]: [ws_bill_customer_sk#43, ws_ext_discount_amt#44, ws_ext_list_price#45 Input [4]: [ws_bill_customer_sk#43, ws_ext_discount_amt#44, ws_ext_list_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_bill_customer_sk#43) -(43) ReusedExchange [Reuses operator id: 83] +(43) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#47, d_year#48] (44) BroadcastHashJoin [codegen id : 19] @@ -338,171 +340,179 @@ Results [2]: [c_customer_id#50 AS customer_id#60, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#60, year_total#61] Condition : (isnotnull(year_total#61) AND (year_total#61 > 0.00)) -(56) Exchange +(56) Project [codegen id : 24] +Output [2]: [customer_id#60 AS customer_id#62, year_total#61 AS year_total#63] Input [2]: [customer_id#60, year_total#61] -Arguments: hashpartitioning(customer_id#60, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(57) Sort [codegen id : 25] -Input [2]: [customer_id#60, year_total#61] -Arguments: [customer_id#60 ASC NULLS FIRST], false, 0 +(57) Exchange +Input [2]: [customer_id#62, year_total#63] +Arguments: hashpartitioning(customer_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(58) Sort [codegen id : 25] +Input [2]: [customer_id#62, year_total#63] +Arguments: [customer_id#62 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 26] +(59) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#60] +Right keys [1]: [customer_id#62] Join type: Inner Join condition: None -(59) Project [codegen id : 26] -Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#61] -Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#60, year_total#61] +(60) Project [codegen id : 26] +Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#63] +Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#62, year_total#63] -(60) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65] +(61) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65] +(62) ColumnarToRow [codegen id : 28] +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(62) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_customer_sk#62) +(63) Filter [codegen id : 28] +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Condition : isnotnull(ws_bill_customer_sk#64) -(63) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#66, d_year#67] +(64) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#68, d_year#69] -(64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] +(65) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ws_sold_date_sk#67] +Right keys [1]: [d_date_sk#68] Join type: Inner Join condition: None -(65) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] -Input [6]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, ws_sold_date_sk#65, d_date_sk#66, d_year#67] +(66) Project [codegen id : 28] +Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#69] +Input [6]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#68, d_year#69] -(66) Exchange -Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] -Arguments: hashpartitioning(ws_bill_customer_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(67) Exchange +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#69] +Arguments: hashpartitioning(ws_bill_customer_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(67) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] -Arguments: [ws_bill_customer_sk#62 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 29] +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#69] +Arguments: [ws_bill_customer_sk#64 ASC NULLS FIRST], false, 0 -(68) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] +(69) ReusedExchange [Reuses operator id: 12] +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -(69) Sort [codegen id : 31] -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Arguments: [c_customer_sk#68 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 31] +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 -(70) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#62] -Right keys [1]: [c_customer_sk#68] +(71) SortMergeJoin [codegen id : 32] +Left keys [1]: [ws_bill_customer_sk#64] +Right keys [1]: [c_customer_sk#70] Join type: Inner Join condition: None -(71) Project [codegen id : 32] -Output [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] -Input [12]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67, c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] - -(72) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67] -Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))] -Aggregate Attributes [1]: [sum#76] -Results [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77] - -(73) Exchange -Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77] -Arguments: hashpartitioning(c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(74) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77] -Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))#59] -Results [2]: [c_customer_id#69 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))#59,18,2) AS year_total#79] - -(75) Exchange -Input [2]: [customer_id#78, year_total#79] -Arguments: hashpartitioning(customer_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(76) Sort [codegen id : 34] -Input [2]: [customer_id#78, year_total#79] -Arguments: [customer_id#78 ASC NULLS FIRST], false, 0 - -(77) SortMergeJoin [codegen id : 35] +(72) Project [codegen id : 32] +Output [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#69] +Input [12]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#69, c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] + +(73) HashAggregate [codegen id : 32] +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#69] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#69] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] +Aggregate Attributes [1]: [sum#78] +Results [9]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#69, sum#79] + +(74) Exchange +Input [9]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#69, sum#79] +Arguments: hashpartitioning(c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(75) HashAggregate [codegen id : 33] +Input [9]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#69, sum#79] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#69] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))#59] +Results [2]: [c_customer_id#71 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))#59,18,2) AS year_total#81] + +(76) Project [codegen id : 33] +Output [2]: [customer_id#80 AS customer_id#82, year_total#81 AS year_total#83] +Input [2]: [customer_id#80, year_total#81] + +(77) Exchange +Input [2]: [customer_id#82, year_total#83] +Arguments: hashpartitioning(customer_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(78) Sort [codegen id : 34] +Input [2]: [customer_id#82, year_total#83] +Arguments: [customer_id#82 ASC NULLS FIRST], false, 0 + +(79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#78] +Right keys [1]: [customer_id#82] Join type: Inner -Join condition: (CASE WHEN (year_total#61 > 0.00) THEN (year_total#79 / year_total#61) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) +Join condition: (CASE WHEN (year_total#63 > 0.00) THEN (year_total#83 / year_total#63) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) -(78) Project [codegen id : 35] +(80) Project [codegen id : 35] Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] -Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#61, customer_id#78, year_total#79] +Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#63, customer_id#82, year_total#83] -(79) TakeOrderedAndProject +(81) TakeOrderedAndProject Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (85) ++- * Filter (84) + +- * ColumnarToRow (83) + +- Scan parquet spark_catalog.default.date_dim (82) -(80) Scan parquet spark_catalog.default.date_dim +(82) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#6, d_year#7] -(82) Filter [codegen id : 1] +(84) Filter [codegen id : 1] Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk#6)) -(83) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (87) -+- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (89) ++- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet spark_catalog.default.date_dim (86) -(84) Scan parquet spark_catalog.default.date_dim +(86) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#26, d_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#26, d_year#27] -(86) Filter [codegen id : 1] +(88) Filter [codegen id : 1] Input [2]: [d_date_sk#26, d_year#27] Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2002)) AND isnotnull(d_date_sk#26)) -(87) BroadcastExchange +(89) BroadcastExchange Input [2]: [d_date_sk#26, d_year#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#25 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt index c4a1eef73183b..3a62a0708a598 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt @@ -99,64 +99,66 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter Exchange [customer_id] #10 WholeStageCodegen (24) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (20) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (19) - Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #4 - InputAdapter - WholeStageCodegen (22) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (23) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - WholeStageCodegen (32) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (28) - Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #9 - InputAdapter - WholeStageCodegen (31) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (32) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (28) + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #9 + InputAdapter + WholeStageCodegen (31) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 2fc3d1975abfd..f5662606c4b4f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) +TakeOrderedAndProject (73) ++- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * BroadcastHashJoin Inner BuildRight (33) : : :- * Filter (16) : : : +- * HashAggregate (15) @@ -37,39 +37,41 @@ TakeOrderedAndProject (71) : : : +- * ColumnarToRow (21) : : : +- Scan parquet spark_catalog.default.store_sales (20) : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * Filter (39) - : : +- * ColumnarToRow (38) - : : +- Scan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) - +- BroadcastExchange (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * Filter (58) - : +- * ColumnarToRow (57) - : +- Scan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (70) + +- * Project (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (56) + : : +- * ColumnarToRow (55) + : : +- Scan parquet spark_catalog.default.customer (54) + : +- BroadcastExchange (60) + : +- * Filter (59) + : +- * ColumnarToRow (58) + : +- Scan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (63) (1) Scan parquet spark_catalog.default.customer @@ -115,7 +117,7 @@ Join condition: None Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 75] +(10) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#14, d_year#15] (11) BroadcastHashJoin [codegen id : 3] @@ -193,7 +195,7 @@ Join condition: None Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -(26) ReusedExchange [Reuses operator id: 79] +(26) ReusedExchange [Reuses operator id: 81] Output [2]: [d_date_sk#34, d_year#35] (27) BroadcastHashJoin [codegen id : 6] @@ -277,7 +279,7 @@ Join condition: None Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -(43) ReusedExchange [Reuses operator id: 75] +(43) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#55, d_year#56] (44) BroadcastHashJoin [codegen id : 10] @@ -312,166 +314,174 @@ Results [2]: [c_customer_id#44 AS customer_id#60, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#60, year_total#61] Condition : (isnotnull(year_total#61) AND (year_total#61 > 0.00)) -(50) BroadcastExchange +(50) Project [codegen id : 11] +Output [2]: [customer_id#60 AS customer_id#62, year_total#61 AS year_total#63] Input [2]: [customer_id#60, year_total#61] + +(51) BroadcastExchange +Input [2]: [customer_id#62, year_total#63] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#60] +Right keys [1]: [customer_id#62] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#61] -Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#60, year_total#61] +(53) Project [codegen id : 16] +Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#63] +Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#62, year_total#63] -(53) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +(54) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#64, c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +(55) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#64, c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71] -(55) Filter [codegen id : 14] -Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] -Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) +(56) Filter [codegen id : 14] +Input [8]: [c_customer_sk#64, c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71] +Condition : (isnotnull(c_customer_sk#64) AND isnotnull(c_customer_id#65)) -(56) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +(57) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#33)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +(58) ColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] -(58) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] -Condition : isnotnull(ws_bill_customer_sk#70) +(59) Filter [codegen id : 12] +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_bill_customer_sk#72) -(59) BroadcastExchange -Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +(60) BroadcastExchange +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#62] -Right keys [1]: [ws_bill_customer_sk#70] +(61) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#64] +Right keys [1]: [ws_bill_customer_sk#72] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] -Input [12]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +(62) Project [codegen id : 14] +Output [10]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] +Input [12]: [c_customer_sk#64, c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#74, d_year#75] +(63) ReusedExchange [Reuses operator id: 81] +Output [2]: [d_date_sk#76, d_year#77] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#73] -Right keys [1]: [d_date_sk#74] +(64) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73, d_date_sk#74, d_year#75] - -(65) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))] -Aggregate Attributes [1]: [sum#76] -Results [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77] - -(66) Exchange -Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))#59] -Results [2]: [c_customer_id#63 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))#59,18,2) AS year_total#79] - -(68) BroadcastExchange -Input [2]: [customer_id#78, year_total#79] +(65) Project [codegen id : 14] +Output [10]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Input [12]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75, d_date_sk#76, d_year#77] + +(66) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Keys [8]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, d_year#77] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#74 - ws_ext_discount_amt#73)))] +Aggregate Attributes [1]: [sum#78] +Results [9]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, d_year#77, sum#79] + +(67) Exchange +Input [9]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, d_year#77, sum#79] +Arguments: hashpartitioning(c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(68) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, d_year#77, sum#79] +Keys [8]: [c_customer_id#65, c_first_name#66, c_last_name#67, c_preferred_cust_flag#68, c_birth_country#69, c_login#70, c_email_address#71, d_year#77] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#74 - ws_ext_discount_amt#73)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#74 - ws_ext_discount_amt#73)))#59] +Results [2]: [c_customer_id#65 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#74 - ws_ext_discount_amt#73)))#59,18,2) AS year_total#81] + +(69) Project [codegen id : 15] +Output [2]: [customer_id#80 AS customer_id#82, year_total#81 AS year_total#83] +Input [2]: [customer_id#80, year_total#81] + +(70) BroadcastExchange +Input [2]: [customer_id#82, year_total#83] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#78] +Right keys [1]: [customer_id#82] Join type: Inner -Join condition: (CASE WHEN (year_total#61 > 0.00) THEN (year_total#79 / year_total#61) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) +Join condition: (CASE WHEN (year_total#63 > 0.00) THEN (year_total#83 / year_total#63) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) -(70) Project [codegen id : 16] +(72) Project [codegen id : 16] Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] -Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#61, customer_id#78, year_total#79] +Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#63, customer_id#82, year_total#83] -(71) TakeOrderedAndProject +(73) TakeOrderedAndProject Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (75) -+- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * Filter (76) + +- * ColumnarToRow (75) + +- Scan parquet spark_catalog.default.date_dim (74) -(72) Scan parquet spark_catalog.default.date_dim +(74) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(74) Filter [codegen id : 1] +(76) Filter [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(75) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (79) -+- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (81) ++- * Filter (80) + +- * ColumnarToRow (79) + +- Scan parquet spark_catalog.default.date_dim (78) -(76) Scan parquet spark_catalog.default.date_dim +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#34, d_year#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(77) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(78) Filter [codegen id : 1] +(80) Filter [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) -(79) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#73 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#33 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index ba743ea45edf7..cf9f215b794d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -70,53 +70,55 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 b4dbb51115112..71b0eb2242e54 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 @@ -1,12 +1,12 @@ == Physical Plan == -TakeOrderedAndProject (90) -+- * BroadcastHashJoin Inner BuildRight (89) - :- * Filter (72) - : +- * HashAggregate (71) - : +- Exchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) +TakeOrderedAndProject (88) ++- * BroadcastHashJoin Inner BuildRight (87) + :- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) : :- * Project (60) : : +- * BroadcastHashJoin Inner BuildRight (59) : : :- * BroadcastHashJoin LeftSemi BuildRight (57) @@ -67,28 +67,26 @@ TakeOrderedAndProject (90) : : : : +- ReusedExchange (44) : : : +- ReusedExchange (47) : : +- ReusedExchange (58) - : +- BroadcastExchange (66) - : +- * BroadcastHashJoin LeftSemi BuildRight (65) - : :- * Filter (63) - : : +- * ColumnarToRow (62) - : : +- Scan parquet spark_catalog.default.item (61) - : +- ReusedExchange (64) - +- BroadcastExchange (88) - +- * Filter (87) - +- * HashAggregate (86) - +- Exchange (85) - +- * HashAggregate (84) - +- * Project (83) - +- * BroadcastHashJoin Inner BuildRight (82) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * BroadcastHashJoin LeftSemi BuildRight (77) - : : :- * Filter (75) - : : : +- * ColumnarToRow (74) - : : : +- Scan parquet spark_catalog.default.store_sales (73) - : : +- ReusedExchange (76) - : +- ReusedExchange (78) - +- ReusedExchange (81) + : +- BroadcastExchange (64) + : +- * Filter (63) + : +- * ColumnarToRow (62) + : +- Scan parquet spark_catalog.default.item (61) + +- BroadcastExchange (86) + +- * Filter (85) + +- * HashAggregate (84) + +- Exchange (83) + +- * HashAggregate (82) + +- * Project (81) + +- * BroadcastHashJoin Inner BuildRight (80) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin LeftSemi BuildRight (75) + : : :- * Filter (73) + : : : +- * ColumnarToRow (72) + : : : +- Scan parquet spark_catalog.default.store_sales (71) + : : +- ReusedExchange (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) Scan parquet spark_catalog.default.store_sales @@ -99,10 +97,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 37] +(2) ColumnarToRow [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 37] +(3) Filter [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -135,7 +133,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 123] +(10) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -185,7 +183,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 123] +(21) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -295,7 +293,7 @@ Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 123] +(44) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#30] (45) BroadcastHashJoin [codegen id : 14] @@ -353,22 +351,22 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 37] +(57) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 114] +(58) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 37] +(59) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 37] +(60) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] @@ -379,59 +377,50 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 36] +(62) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(63) Filter [codegen id : 36] +(63) Filter [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40)) -(64) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#35] - -(65) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [i_item_sk#37] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(66) BroadcastExchange +(64) BroadcastExchange Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(67) BroadcastHashJoin [codegen id : 37] +(65) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(68) Project [codegen id : 37] +(66) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(69) HashAggregate [codegen id : 37] +(67) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(70) Exchange +(68) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(71) HashAggregate [codegen id : 76] +(69) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(72) Filter [codegen id : 76] +(70) Filter [codegen id : 42] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(73) Scan parquet spark_catalog.default.store_sales +(71) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -439,285 +428,285 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(74) ColumnarToRow [codegen id : 74] +(72) ColumnarToRow [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(75) Filter [codegen id : 74] +(73) Filter [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) -(76) ReusedExchange [Reuses operator id: 56] +(74) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#35] -(77) BroadcastHashJoin [codegen id : 74] +(75) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(78) ReusedExchange [Reuses operator id: 128] +(76) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#59] -(79) BroadcastHashJoin [codegen id : 74] +(77) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(80) Project [codegen id : 74] +(78) Project [codegen id : 40] Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56] Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59] -(81) ReusedExchange [Reuses operator id: 66] +(79) ReusedExchange [Reuses operator id: 64] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(82) BroadcastHashJoin [codegen id : 74] +(80) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(83) Project [codegen id : 74] +(81) Project [codegen id : 40] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(84) HashAggregate [codegen id : 74] +(82) HashAggregate [codegen id : 40] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] -(85) Exchange +(83) Exchange Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(86) HashAggregate [codegen id : 75] +(84) HashAggregate [codegen id : 41] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(87) Filter [codegen id : 75] +(85) Filter [codegen id : 41] Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(88) BroadcastExchange +(86) BroadcastExchange Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13] -(89) BroadcastHashJoin [codegen id : 76] +(87) BroadcastHashJoin [codegen id : 42] Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner Join condition: None -(90) TakeOrderedAndProject +(88) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: 100, [i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (109) -+- Exchange (108) - +- * HashAggregate (107) - +- Union (106) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * ColumnarToRow (92) - : : +- Scan parquet spark_catalog.default.store_sales (91) - : +- ReusedExchange (93) - :- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * ColumnarToRow (97) - : : +- Scan parquet spark_catalog.default.catalog_sales (96) - : +- ReusedExchange (98) - +- * Project (105) - +- * BroadcastHashJoin Inner BuildRight (104) - :- * ColumnarToRow (102) - : +- Scan parquet spark_catalog.default.web_sales (101) - +- ReusedExchange (103) - - -(91) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (107) ++- Exchange (106) + +- * HashAggregate (105) + +- Union (104) + :- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * ColumnarToRow (90) + : : +- Scan parquet spark_catalog.default.store_sales (89) + : +- ReusedExchange (91) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * ColumnarToRow (95) + : : +- Scan parquet spark_catalog.default.catalog_sales (94) + : +- ReusedExchange (96) + +- * Project (103) + +- * BroadcastHashJoin Inner BuildRight (102) + :- * ColumnarToRow (100) + : +- Scan parquet spark_catalog.default.web_sales (99) + +- ReusedExchange (101) + + +(89) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 2] +(90) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(93) ReusedExchange [Reuses operator id: 123] +(91) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#78] -(94) BroadcastHashJoin [codegen id : 2] +(92) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(95) Project [codegen id : 2] +(93) Project [codegen id : 2] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] -(96) Scan parquet spark_catalog.default.catalog_sales +(94) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 4] +(95) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -(98) ReusedExchange [Reuses operator id: 123] +(96) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#84] -(99) BroadcastHashJoin [codegen id : 4] +(97) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(100) Project [codegen id : 4] +(98) Project [codegen id : 4] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] -(101) Scan parquet spark_catalog.default.web_sales +(99) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 6] +(100) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(103) ReusedExchange [Reuses operator id: 123] +(101) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#90] -(104) BroadcastHashJoin [codegen id : 6] +(102) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(105) Project [codegen id : 6] +(103) Project [codegen id : 6] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] -(106) Union +(104) Union -(107) HashAggregate [codegen id : 7] +(105) HashAggregate [codegen id : 7] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [2]: [sum#93, count#94] Results [2]: [sum#95, count#96] -(108) Exchange +(106) Exchange Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(109) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 8] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] -Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 94 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (114) -+- * Project (113) - +- * Filter (112) - +- * ColumnarToRow (111) - +- Scan parquet spark_catalog.default.date_dim (110) +BroadcastExchange (112) ++- * Project (111) + +- * Filter (110) + +- * ColumnarToRow (109) + +- Scan parquet spark_catalog.default.date_dim (108) -(110) Scan parquet spark_catalog.default.date_dim +(108) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#36, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 1] +(109) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] -(112) Filter [codegen id : 1] +(110) Filter [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36)) -(113) Project [codegen id : 1] +(111) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [2]: [d_date_sk#36, d_week_seq#99] -(114) BroadcastExchange +(112) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:6 Hosting operator id = 112 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* Project (118) -+- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet spark_catalog.default.date_dim (115) +Subquery:6 Hosting operator id = 110 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* Project (116) ++- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet spark_catalog.default.date_dim (113) -(115) Scan parquet spark_catalog.default.date_dim +(113) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -(117) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) -(118) Project [codegen id : 1] +(116) Project [codegen id : 1] Output [1]: [d_week_seq#102] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (123) -+- * Project (122) - +- * Filter (121) - +- * ColumnarToRow (120) - +- Scan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (121) ++- * Project (120) + +- * Filter (119) + +- * ColumnarToRow (118) + +- Scan parquet spark_catalog.default.date_dim (117) -(119) Scan parquet spark_catalog.default.date_dim +(117) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] +(118) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] -(121) Filter [codegen id : 1] +(119) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#13)) -(122) Project [codegen id : 1] +(120) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#106] -(123) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] @@ -725,60 +714,60 @@ Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN d Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 73 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (128) -+- * Project (127) - +- * Filter (126) - +- * ColumnarToRow (125) - +- Scan parquet spark_catalog.default.date_dim (124) +Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (126) ++- * Project (125) + +- * Filter (124) + +- * ColumnarToRow (123) + +- Scan parquet spark_catalog.default.date_dim (122) -(124) Scan parquet spark_catalog.default.date_dim +(122) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#59, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(125) ColumnarToRow [codegen id : 1] +(123) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] -(126) Filter [codegen id : 1] +(124) Filter [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59)) -(127) Project [codegen id : 1] +(125) Project [codegen id : 1] Output [1]: [d_date_sk#59] Input [2]: [d_date_sk#59, d_week_seq#107] -(128) BroadcastExchange +(126) BroadcastExchange Input [1]: [d_date_sk#59] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -Subquery:12 Hosting operator id = 126 Hosting Expression = Subquery scalar-subquery#108, [id=#109] -* Project (132) -+- * Filter (131) - +- * ColumnarToRow (130) - +- Scan parquet spark_catalog.default.date_dim (129) +Subquery:12 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* Project (130) ++- * Filter (129) + +- * ColumnarToRow (128) + +- Scan parquet spark_catalog.default.date_dim (127) -(129) Scan parquet spark_catalog.default.date_dim +(127) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(130) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -(131) Filter [codegen id : 1] +(129) Filter [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16)) -(132) Project [codegen id : 1] +(130) Project [codegen id : 1] Output [1]: [d_week_seq#110] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] 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 0d6f76d6e73bb..005d0cc8cfb42 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 @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (76) + WholeStageCodegen (42) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 @@ -41,7 +41,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (37) + WholeStageCodegen (20) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -170,23 +170,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (36) - BroadcastHashJoin [i_item_sk,ss_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] - InputAdapter - ReusedExchange [ss_item_sk] #3 + WholeStageCodegen (19) + 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] InputAdapter BroadcastExchange #15 - WholeStageCodegen (75) + WholeStageCodegen (41) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (74) + WholeStageCodegen (40) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] 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 614f0a6da6f1f..6f28a5b6f2723 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 @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * BroadcastHashJoin Inner BuildRight (83) - :- * Filter (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) +TakeOrderedAndProject (82) ++- * BroadcastHashJoin Inner BuildRight (81) + :- * Filter (64) + : +- * HashAggregate (63) + : +- Exchange (62) + : +- * HashAggregate (61) + : +- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (57) + : : +- * BroadcastHashJoin Inner BuildRight (56) : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) @@ -20,69 +20,67 @@ TakeOrderedAndProject (84) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (57) - : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : :- * Filter (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet spark_catalog.default.item (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (60) - +- BroadcastExchange (82) - +- * Filter (81) - +- * HashAggregate (80) - +- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * BroadcastHashJoin Inner BuildRight (76) - :- * Project (74) - : +- * BroadcastHashJoin Inner BuildRight (73) - : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : :- * Filter (69) - : : : +- * ColumnarToRow (68) - : : : +- Scan parquet spark_catalog.default.store_sales (67) - : : +- ReusedExchange (70) - : +- ReusedExchange (72) - +- ReusedExchange (75) + : : : +- * HashAggregate (46) + : : : +- Exchange (45) + : : : +- * HashAggregate (44) + : : : +- * Project (43) + : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : :- * Project (40) + : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : :- * Filter (9) + : : : : : +- * ColumnarToRow (8) + : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : +- BroadcastExchange (38) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) + : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- BroadcastExchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (41) + : : +- BroadcastExchange (55) + : : +- * Filter (54) + : : +- * ColumnarToRow (53) + : : +- Scan parquet spark_catalog.default.item (52) + : +- ReusedExchange (58) + +- BroadcastExchange (80) + +- * Filter (79) + +- * HashAggregate (78) + +- Exchange (77) + +- * HashAggregate (76) + +- * Project (75) + +- * BroadcastHashJoin Inner BuildRight (74) + :- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * BroadcastHashJoin LeftSemi BuildRight (69) + : : :- * Filter (67) + : : : +- * ColumnarToRow (66) + : : : +- Scan parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (68) + : +- ReusedExchange (70) + +- ReusedExchange (73) (1) Scan parquet spark_catalog.default.store_sales @@ -93,10 +91,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -122,10 +120,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -136,10 +134,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -186,7 +184,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(22) ReusedExchange [Reuses operator id: 117] +(22) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 3] @@ -203,127 +201,127 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 4] +(26) BroadcastHashJoin [codegen id : 7] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(27) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(28) ColumnarToRow [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(29) Filter [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#24) -(30) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [ws_item_sk#24] +Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] +Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] +Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) Exchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(33) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#30] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] +(34) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#25] +Right keys [1]: [d_date_sk#30] +Join type: Inner +Join condition: None -(36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(35) Project [codegen id : 6] +Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] -(37) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +(36) BroadcastExchange +Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] -(38) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) +(37) BroadcastHashJoin [codegen id : 7] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] +Join type: LeftSemi +Join condition: None -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(38) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] +(39) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(40) Project [codegen id : 9] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(42) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#34] +(41) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#31] -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] +(42) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] +(43) Project [codegen id : 9] +Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] + +(44) HashAggregate [codegen id : 9] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] -(45) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +(45) Exchange +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None +(46) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] (47) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] +Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] (50) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -336,72 +334,63 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 23] +(53) ColumnarToRow [codegen id : 12] Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(54) Filter [codegen id : 23] +(54) Filter [codegen id : 12] Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) -(55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#35] - -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(57) BroadcastExchange +(55) BroadcastExchange Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(57) Project [codegen id : 14] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(60) ReusedExchange [Reuses operator id: 108] +(58) ReusedExchange [Reuses operator id: 106] Output [1]: [d_date_sk#40] -(61) BroadcastHashJoin [codegen id : 25] +(59) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(60) Project [codegen id : 14] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] -(63) HashAggregate [codegen id : 25] +(61) HashAggregate [codegen id : 14] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -(64) Exchange +(62) Exchange Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(65) HashAggregate [codegen id : 52] +(63) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(66) Filter [codegen id : 52] +(64) Filter [codegen id : 30] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.store_sales +(65) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -409,346 +398,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 50] +(66) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(69) Filter [codegen id : 50] +(67) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) -(70) ReusedExchange [Reuses operator id: 50] +(68) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#35] -(71) BroadcastHashJoin [codegen id : 50] +(69) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(72) ReusedExchange [Reuses operator id: 57] +(70) ReusedExchange [Reuses operator id: 55] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(73) BroadcastHashJoin [codegen id : 50] +(71) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(74) Project [codegen id : 50] +(72) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) ReusedExchange [Reuses operator id: 122] +(73) ReusedExchange [Reuses operator id: 120] Output [1]: [d_date_sk#63] -(76) BroadcastHashJoin [codegen id : 50] +(74) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(77) Project [codegen id : 50] +(75) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -(78) HashAggregate [codegen id : 50] +(76) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(79) Exchange +(77) Exchange Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(80) HashAggregate [codegen id : 51] +(78) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(81) Filter [codegen id : 51] +(79) Filter [codegen id : 29] Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(82) BroadcastExchange +(80) BroadcastExchange Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(83) BroadcastHashJoin [codegen id : 52] +(81) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join type: Inner Join condition: None -(84) TakeOrderedAndProject +(82) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- Scan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- Scan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- Scan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) - - -(85) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (101) ++- Exchange (100) + +- * HashAggregate (99) + +- Union (98) + :- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * ColumnarToRow (84) + : : +- Scan parquet spark_catalog.default.store_sales (83) + : +- ReusedExchange (85) + :- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * ColumnarToRow (89) + : : +- Scan parquet spark_catalog.default.catalog_sales (88) + : +- ReusedExchange (90) + +- * Project (97) + +- * BroadcastHashJoin Inner BuildRight (96) + :- * ColumnarToRow (94) + : +- Scan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (95) + + +(83) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] +(84) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(87) ReusedExchange [Reuses operator id: 117] +(85) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#78] -(88) BroadcastHashJoin [codegen id : 2] +(86) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(89) Project [codegen id : 2] +(87) Project [codegen id : 2] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] -(90) Scan parquet spark_catalog.default.catalog_sales +(88) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] +(89) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -(92) ReusedExchange [Reuses operator id: 117] +(90) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#84] -(93) BroadcastHashJoin [codegen id : 4] +(91) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(94) Project [codegen id : 4] +(92) Project [codegen id : 4] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] -(95) Scan parquet spark_catalog.default.web_sales +(93) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] +(94) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(97) ReusedExchange [Reuses operator id: 117] +(95) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#90] -(98) BroadcastHashJoin [codegen id : 6] +(96) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(99) Project [codegen id : 6] +(97) Project [codegen id : 6] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] -(100) Union +(98) Union -(101) HashAggregate [codegen id : 7] +(99) HashAggregate [codegen id : 7] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [2]: [sum#93, count#94] Results [2]: [sum#95, count#96] -(102) Exchange +(100) Exchange Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(103) HashAggregate [codegen id : 8] +(101) HashAggregate [codegen id : 8] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 88 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * Project (107) - +- * Filter (106) - +- * ColumnarToRow (105) - +- Scan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (106) ++- * Project (105) + +- * Filter (104) + +- * ColumnarToRow (103) + +- Scan parquet spark_catalog.default.date_dim (102) -(104) Scan parquet spark_catalog.default.date_dim +(102) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#40, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_week_seq#99] -(106) Filter [codegen id : 1] +(104) Filter [codegen id : 1] Input [2]: [d_date_sk#40, d_week_seq#99] Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#40)) -(107) Project [codegen id : 1] +(105) Project [codegen id : 1] Output [1]: [d_date_sk#40] Input [2]: [d_date_sk#40, d_week_seq#99] -(108) BroadcastExchange +(106) BroadcastExchange Input [1]: [d_date_sk#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 106 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* Project (112) -+- * Filter (111) - +- * ColumnarToRow (110) - +- Scan parquet spark_catalog.default.date_dim (109) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* Project (110) ++- * Filter (109) + +- * ColumnarToRow (108) + +- Scan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim +(107) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(110) ColumnarToRow [codegen id : 1] +(108) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -(111) Filter [codegen id : 1] +(109) Filter [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) -(112) Project [codegen id : 1] +(110) Project [codegen id : 1] Output [1]: [d_week_seq#102] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * Project (116) - +- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (115) ++- * Project (114) + +- * Filter (113) + +- * ColumnarToRow (112) + +- Scan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#106] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#106] +(112) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#106] -(115) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#106] -Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#24)) +(113) Filter [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#106] +Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#31)) -(116) Project [codegen id : 1] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#106] +(114) Project [codegen id : 1] +Output [1]: [d_date_sk#31] +Input [2]: [d_date_sk#31, d_year#106] -(117) BroadcastExchange -Input [1]: [d_date_sk#24] +(115) BroadcastExchange +Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (122) -+- * Project (121) - +- * Filter (120) - +- * ColumnarToRow (119) - +- Scan parquet spark_catalog.default.date_dim (118) +Subquery:11 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (120) ++- * Project (119) + +- * Filter (118) + +- * ColumnarToRow (117) + +- Scan parquet spark_catalog.default.date_dim (116) -(118) Scan parquet spark_catalog.default.date_dim +(116) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#63, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(119) ColumnarToRow [codegen id : 1] +(117) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#63, d_week_seq#107] -(120) Filter [codegen id : 1] +(118) Filter [codegen id : 1] Input [2]: [d_date_sk#63, d_week_seq#107] Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#63)) -(121) Project [codegen id : 1] +(119) Project [codegen id : 1] Output [1]: [d_date_sk#63] Input [2]: [d_date_sk#63, d_week_seq#107] -(122) BroadcastExchange +(120) BroadcastExchange Input [1]: [d_date_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:12 Hosting operator id = 120 Hosting Expression = Subquery scalar-subquery#108, [id=#109] -* Project (126) -+- * Filter (125) - +- * ColumnarToRow (124) - +- Scan parquet spark_catalog.default.date_dim (123) +Subquery:12 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* Project (124) ++- * Filter (123) + +- * ColumnarToRow (122) + +- Scan parquet spark_catalog.default.date_dim (121) -(123) Scan parquet spark_catalog.default.date_dim +(121) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(124) ColumnarToRow [codegen id : 1] +(122) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -(125) Filter [codegen id : 1] +(123) Filter [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16)) -(126) Project [codegen id : 1] +(124) Project [codegen id : 1] Output [1]: [d_week_seq#110] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] 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 fb2e53c91965a..825bb2d20a585 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 @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 @@ -41,7 +41,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -79,31 +79,31 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - 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 #7 - WholeStageCodegen (4) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + 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 #7 + WholeStageCodegen (7) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -130,45 +130,42 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_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] - InputAdapter - ReusedExchange [ss_item_sk] #3 + WholeStageCodegen (12) + 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] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] 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 5910703da16ff..0358a4084c20a 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 @@ -1,19 +1,19 @@ == Physical Plan == -TakeOrderedAndProject (131) -+- * HashAggregate (130) - +- Exchange (129) - +- * HashAggregate (128) - +- Union (127) - :- * HashAggregate (106) - : +- Exchange (105) - : +- * HashAggregate (104) - : +- Union (103) - : :- * Filter (72) - : : +- * HashAggregate (71) - : : +- Exchange (70) - : : +- * HashAggregate (69) - : : +- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) +TakeOrderedAndProject (126) ++- * HashAggregate (125) + +- Exchange (124) + +- * HashAggregate (123) + +- Union (122) + :- * HashAggregate (101) + : +- Exchange (100) + : +- * HashAggregate (99) + : +- Union (98) + : :- * Filter (67) + : : +- * HashAggregate (66) + : : +- Exchange (65) + : : +- * HashAggregate (64) + : : +- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) : : :- * Project (60) : : : +- * BroadcastHashJoin Inner BuildRight (59) : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) @@ -74,42 +74,42 @@ TakeOrderedAndProject (131) : : : : : +- ReusedExchange (44) : : : : +- ReusedExchange (47) : : : +- ReusedExchange (58) - : : +- BroadcastExchange (66) - : : +- * BroadcastHashJoin LeftSemi BuildRight (65) - : : :- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet spark_catalog.default.item (61) - : : +- ReusedExchange (64) - : :- * Filter (87) - : : +- * HashAggregate (86) - : : +- Exchange (85) - : : +- * HashAggregate (84) - : : +- * Project (83) - : : +- * BroadcastHashJoin Inner BuildRight (82) - : : :- * Project (80) - : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (77) - : : : : :- * Filter (75) - : : : : : +- * ColumnarToRow (74) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (73) - : : : : +- ReusedExchange (76) - : : : +- ReusedExchange (78) - : : +- ReusedExchange (81) - : +- * Filter (102) - : +- * HashAggregate (101) - : +- Exchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * BroadcastHashJoin LeftSemi BuildRight (92) - : : : :- * Filter (90) - : : : : +- * ColumnarToRow (89) - : : : : +- Scan parquet spark_catalog.default.web_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) + : : +- ReusedExchange (61) + : :- * Filter (82) + : : +- * HashAggregate (81) + : : +- Exchange (80) + : : +- * HashAggregate (79) + : : +- * Project (78) + : : +- * BroadcastHashJoin Inner BuildRight (77) + : : :- * Project (75) + : : : +- * BroadcastHashJoin Inner BuildRight (74) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (72) + : : : : :- * Filter (70) + : : : : : +- * ColumnarToRow (69) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (68) + : : : : +- ReusedExchange (71) + : : : +- ReusedExchange (73) + : : +- ReusedExchange (76) + : +- * Filter (97) + : +- * HashAggregate (96) + : +- Exchange (95) + : +- * HashAggregate (94) + : +- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * Project (90) + : : +- * BroadcastHashJoin Inner BuildRight (89) + : : :- * BroadcastHashJoin LeftSemi BuildRight (87) + : : : :- * Filter (85) + : : : : +- * ColumnarToRow (84) + : : : : +- Scan parquet spark_catalog.default.web_sales (83) + : : : +- ReusedExchange (86) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- * HashAggregate (106) + : +- Exchange (105) + : +- * HashAggregate (104) + : +- * HashAggregate (103) + : +- ReusedExchange (102) :- * HashAggregate (111) : +- Exchange (110) : +- * HashAggregate (109) @@ -120,16 +120,11 @@ TakeOrderedAndProject (131) : +- * HashAggregate (114) : +- * HashAggregate (113) : +- ReusedExchange (112) - :- * HashAggregate (121) - : +- Exchange (120) - : +- * HashAggregate (119) - : +- * HashAggregate (118) - : +- ReusedExchange (117) - +- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- ReusedExchange (122) + +- * HashAggregate (121) + +- Exchange (120) + +- * HashAggregate (119) + +- * HashAggregate (118) + +- ReusedExchange (117) (1) Scan parquet spark_catalog.default.store_sales @@ -140,10 +135,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 37] +(2) ColumnarToRow [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 37] +(3) Filter [codegen id : 20] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -176,7 +171,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 165] +(10) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -226,7 +221,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 165] +(21) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -336,7 +331,7 @@ Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 165] +(44) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#30] (45) BroadcastHashJoin [codegen id : 14] @@ -394,85 +389,61 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 37] +(57) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 160] +(58) ReusedExchange [Reuses operator id: 155] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 37] +(59) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 37] +(60) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] -(61) Scan parquet spark_catalog.default.item +(61) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) ColumnarToRow [codegen id : 36] -Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] - -(63) Filter [codegen id : 36] -Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -Condition : isnotnull(i_item_sk#37) - -(64) ReusedExchange [Reuses operator id: 56] -Output [1]: [ss_item_sk#35] - -(65) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [i_item_sk#37] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(66) BroadcastExchange -Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(67) BroadcastHashJoin [codegen id : 37] +(62) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(68) Project [codegen id : 37] +(63) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(69) HashAggregate [codegen id : 37] +(64) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(70) Exchange +(65) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(71) HashAggregate [codegen id : 38] +(66) HashAggregate [codegen id : 21] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(72) Filter [codegen id : 38] +(67) Filter [codegen id : 21] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(73) Scan parquet spark_catalog.default.catalog_sales +(68) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -480,71 +451,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(74) ColumnarToRow [codegen id : 75] +(69) ColumnarToRow [codegen id : 41] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(75) Filter [codegen id : 75] +(70) Filter [codegen id : 41] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Condition : isnotnull(cs_item_sk#54) -(76) ReusedExchange [Reuses operator id: 56] +(71) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#35] -(77) BroadcastHashJoin [codegen id : 75] +(72) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(78) ReusedExchange [Reuses operator id: 160] +(73) ReusedExchange [Reuses operator id: 155] Output [1]: [d_date_sk#58] -(79) BroadcastHashJoin [codegen id : 75] +(74) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_sold_date_sk#57] Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(80) Project [codegen id : 75] +(75) Project [codegen id : 41] Output [3]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56] Input [5]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, d_date_sk#58] -(81) ReusedExchange [Reuses operator id: 66] +(76) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(82) BroadcastHashJoin [codegen id : 75] +(77) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(83) Project [codegen id : 75] +(78) Project [codegen id : 41] Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(84) HashAggregate [codegen id : 75] +(79) HashAggregate [codegen id : 41] Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -(85) Exchange +(80) Exchange Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(86) HashAggregate [codegen id : 76] +(81) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70] Results [6]: [catalog AS channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73] -(87) Filter [codegen id : 76] +(82) Filter [codegen id : 42] Input [6]: [channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(88) Scan parquet spark_catalog.default.web_sales +(83) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] @@ -552,443 +523,443 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 113] +(84) ColumnarToRow [codegen id : 62] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -(90) Filter [codegen id : 113] +(85) Filter [codegen id : 62] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Condition : isnotnull(ws_item_sk#74) -(91) ReusedExchange [Reuses operator id: 56] +(86) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#35] -(92) BroadcastHashJoin [codegen id : 113] +(87) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#74] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(93) ReusedExchange [Reuses operator id: 160] +(88) ReusedExchange [Reuses operator id: 155] Output [1]: [d_date_sk#78] -(94) BroadcastHashJoin [codegen id : 113] +(89) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(95) Project [codegen id : 113] +(90) Project [codegen id : 62] Output [3]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76] Input [5]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, d_date_sk#78] -(96) ReusedExchange [Reuses operator id: 66] +(91) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(97) BroadcastHashJoin [codegen id : 113] +(92) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#74] Right keys [1]: [i_item_sk#79] Join type: Inner Join condition: None -(98) Project [codegen id : 113] +(93) Project [codegen id : 62] Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82] Input [7]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(99) HashAggregate [codegen id : 113] +(94) HashAggregate [codegen id : 62] Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82] Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -(100) Exchange +(95) Exchange Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(101) HashAggregate [codegen id : 114] +(96) HashAggregate [codegen id : 63] Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90] Results [6]: [web AS channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93] -(102) Filter [codegen id : 114] +(97) Filter [codegen id : 63] Input [6]: [channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(103) Union +(98) Union -(104) HashAggregate [codegen id : 115] +(99) HashAggregate [codegen id : 64] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] Aggregate Attributes [3]: [sum#94, isEmpty#95, sum#96] Results [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(105) Exchange +(100) Exchange Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(106) HashAggregate [codegen id : 116] +(101) HashAggregate [codegen id : 65] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(107) ReusedExchange [Reuses operator id: 105] +(102) ReusedExchange [Reuses operator id: 100] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(108) HashAggregate [codegen id : 232] +(103) HashAggregate [codegen id : 130] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [5]: [channel#49, i_brand_id#38, i_class_id#39, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(109) HashAggregate [codegen id : 232] +(104) HashAggregate [codegen id : 130] Input [5]: [channel#49, i_brand_id#38, i_class_id#39, sum_sales#102, number_sales#103] Keys [3]: [channel#49, i_brand_id#38, i_class_id#39] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109] -(110) Exchange +(105) Exchange Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109] -Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(111) HashAggregate [codegen id : 233] +(106) HashAggregate [codegen id : 131] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109] Keys [3]: [channel#49, i_brand_id#38, i_class_id#39] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#110, sum(number_sales#103)#111] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, null AS i_category_id#112, sum(sum_sales#102)#110 AS sum(sum_sales)#113, sum(number_sales#103)#111 AS sum(number_sales)#114] -(112) ReusedExchange [Reuses operator id: 105] +(107) ReusedExchange [Reuses operator id: 100] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(113) HashAggregate [codegen id : 349] +(108) HashAggregate [codegen id : 196] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [4]: [channel#49, i_brand_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(114) HashAggregate [codegen id : 349] +(109) HashAggregate [codegen id : 196] Input [4]: [channel#49, i_brand_id#38, sum_sales#102, number_sales#103] Keys [2]: [channel#49, i_brand_id#38] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] Results [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120] -(115) Exchange +(110) Exchange Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(116) HashAggregate [codegen id : 350] +(111) HashAggregate [codegen id : 197] Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120] Keys [2]: [channel#49, i_brand_id#38] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#121, sum(number_sales#103)#122] Results [6]: [channel#49, i_brand_id#38, null AS i_class_id#123, null AS i_category_id#124, sum(sum_sales#102)#121 AS sum(sum_sales)#125, sum(number_sales#103)#122 AS sum(number_sales)#126] -(117) ReusedExchange [Reuses operator id: 105] +(112) ReusedExchange [Reuses operator id: 100] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(118) HashAggregate [codegen id : 466] +(113) HashAggregate [codegen id : 262] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(119) HashAggregate [codegen id : 466] +(114) HashAggregate [codegen id : 262] Input [3]: [channel#49, sum_sales#102, number_sales#103] Keys [1]: [channel#49] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129] Results [4]: [channel#49, sum#130, isEmpty#131, sum#132] -(120) Exchange +(115) Exchange Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(121) HashAggregate [codegen id : 467] +(116) HashAggregate [codegen id : 263] Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] Keys [1]: [channel#49] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#133, sum(number_sales#103)#134] Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null AS i_category_id#137, sum(sum_sales#102)#133 AS sum(sum_sales)#138, sum(number_sales#103)#134 AS sum(number_sales)#139] -(122) ReusedExchange [Reuses operator id: 105] +(117) ReusedExchange [Reuses operator id: 100] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(123) HashAggregate [codegen id : 583] +(118) HashAggregate [codegen id : 328] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(124) HashAggregate [codegen id : 583] +(119) HashAggregate [codegen id : 328] Input [2]: [sum_sales#102, number_sales#103] Keys: [] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#140, isEmpty#141, sum#142] Results [3]: [sum#143, isEmpty#144, sum#145] -(125) Exchange +(120) Exchange Input [3]: [sum#143, isEmpty#144, sum#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] -(126) HashAggregate [codegen id : 584] +(121) HashAggregate [codegen id : 329] Input [3]: [sum#143, isEmpty#144, sum#145] Keys: [] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#146, sum(number_sales#103)#147] Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#150, null AS i_category_id#151, sum(sum_sales#102)#146 AS sum(sum_sales)#152, sum(number_sales#103)#147 AS sum(number_sales)#153] -(127) Union +(122) Union -(128) HashAggregate [codegen id : 585] +(123) HashAggregate [codegen id : 330] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] -(129) Exchange +(124) Exchange Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] -Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=19] +Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=18] -(130) HashAggregate [codegen id : 586] +(125) HashAggregate [codegen id : 331] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] -(131) TakeOrderedAndProject +(126) TakeOrderedAndProject Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] ===== Subqueries ===== -Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (150) -+- Exchange (149) - +- * HashAggregate (148) - +- Union (147) +Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (145) ++- Exchange (144) + +- * HashAggregate (143) + +- Union (142) + :- * Project (131) + : +- * BroadcastHashJoin Inner BuildRight (130) + : :- * ColumnarToRow (128) + : : +- Scan parquet spark_catalog.default.store_sales (127) + : +- ReusedExchange (129) :- * Project (136) : +- * BroadcastHashJoin Inner BuildRight (135) : :- * ColumnarToRow (133) - : : +- Scan parquet spark_catalog.default.store_sales (132) + : : +- Scan parquet spark_catalog.default.catalog_sales (132) : +- ReusedExchange (134) - :- * Project (141) - : +- * BroadcastHashJoin Inner BuildRight (140) - : :- * ColumnarToRow (138) - : : +- Scan parquet spark_catalog.default.catalog_sales (137) - : +- ReusedExchange (139) - +- * Project (146) - +- * BroadcastHashJoin Inner BuildRight (145) - :- * ColumnarToRow (143) - : +- Scan parquet spark_catalog.default.web_sales (142) - +- ReusedExchange (144) - - -(132) Scan parquet spark_catalog.default.store_sales + +- * Project (141) + +- * BroadcastHashJoin Inner BuildRight (140) + :- * ColumnarToRow (138) + : +- Scan parquet spark_catalog.default.web_sales (137) + +- ReusedExchange (139) + + +(127) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] +(128) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] -(134) ReusedExchange [Reuses operator id: 165] +(129) ReusedExchange [Reuses operator id: 160] Output [1]: [d_date_sk#157] -(135) BroadcastHashJoin [codegen id : 2] +(130) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#156] Right keys [1]: [d_date_sk#157] Join type: Inner Join condition: None -(136) Project [codegen id : 2] +(131) Project [codegen id : 2] Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159] Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157] -(137) Scan parquet spark_catalog.default.catalog_sales +(132) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 4] +(133) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] -(139) ReusedExchange [Reuses operator id: 155] +(134) ReusedExchange [Reuses operator id: 150] Output [1]: [d_date_sk#164] -(140) BroadcastHashJoin [codegen id : 4] +(135) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#162] Right keys [1]: [d_date_sk#164] Join type: Inner Join condition: None -(141) Project [codegen id : 4] +(136) Project [codegen id : 4] Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166] Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164] -(142) Scan parquet spark_catalog.default.web_sales +(137) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)] ReadSchema: struct -(143) ColumnarToRow [codegen id : 6] +(138) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] -(144) ReusedExchange [Reuses operator id: 155] +(139) ReusedExchange [Reuses operator id: 150] Output [1]: [d_date_sk#170] -(145) BroadcastHashJoin [codegen id : 6] +(140) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#169] Right keys [1]: [d_date_sk#170] Join type: Inner Join condition: None -(146) Project [codegen id : 6] +(141) Project [codegen id : 6] Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172] Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170] -(147) Union +(142) Union -(148) HashAggregate [codegen id : 7] +(143) HashAggregate [codegen id : 7] Input [2]: [quantity#158, list_price#159] Keys: [] Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] Aggregate Attributes [2]: [sum#173, count#174] Results [2]: [sum#175, count#176] -(149) Exchange +(144) Exchange Input [2]: [sum#175, count#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19] -(150) HashAggregate [codegen id : 8] +(145) HashAggregate [codegen id : 8] Input [2]: [sum#175, count#176] Keys: [] Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177] Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 127 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 -BroadcastExchange (155) -+- * Project (154) - +- * Filter (153) - +- * ColumnarToRow (152) - +- Scan parquet spark_catalog.default.date_dim (151) +Subquery:3 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 +BroadcastExchange (150) ++- * Project (149) + +- * Filter (148) + +- * ColumnarToRow (147) + +- Scan parquet spark_catalog.default.date_dim (146) -(151) Scan parquet spark_catalog.default.date_dim +(146) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#164, d_year#179] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(152) ColumnarToRow [codegen id : 1] +(147) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#164, d_year#179] -(153) Filter [codegen id : 1] +(148) Filter [codegen id : 1] Input [2]: [d_date_sk#164, d_year#179] Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#164)) -(154) Project [codegen id : 1] +(149) Project [codegen id : 1] Output [1]: [d_date_sk#164] Input [2]: [d_date_sk#164, d_year#179] -(155) BroadcastExchange +(150) BroadcastExchange Input [1]: [d_date_sk#164] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] -Subquery:4 Hosting operator id = 142 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 +Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (160) -+- * Project (159) - +- * Filter (158) - +- * ColumnarToRow (157) - +- Scan parquet spark_catalog.default.date_dim (156) +BroadcastExchange (155) ++- * Project (154) + +- * Filter (153) + +- * ColumnarToRow (152) + +- Scan parquet spark_catalog.default.date_dim (151) -(156) Scan parquet spark_catalog.default.date_dim +(151) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#36, d_year#180, d_moy#181] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(157) ColumnarToRow [codegen id : 1] +(152) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#36, d_year#180, d_moy#181] -(158) Filter [codegen id : 1] +(153) Filter [codegen id : 1] Input [3]: [d_date_sk#36, d_year#180, d_moy#181] Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 = 2000)) AND (d_moy#181 = 11)) AND isnotnull(d_date_sk#36)) -(159) Project [codegen id : 1] +(154) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [3]: [d_date_sk#36, d_year#180, d_moy#181] -(160) BroadcastExchange +(155) BroadcastExchange Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (165) -+- * Project (164) - +- * Filter (163) - +- * ColumnarToRow (162) - +- Scan parquet spark_catalog.default.date_dim (161) +BroadcastExchange (160) ++- * Project (159) + +- * Filter (158) + +- * ColumnarToRow (157) + +- Scan parquet spark_catalog.default.date_dim (156) -(161) Scan parquet spark_catalog.default.date_dim +(156) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#182] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(162) ColumnarToRow [codegen id : 1] +(157) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#182] -(163) Filter [codegen id : 1] +(158) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#182] Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#13)) -(164) Project [codegen id : 1] +(159) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#182] -(165) BroadcastExchange +(160) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22] Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:11 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 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 b93f046fa0a81..f9287bdb7893b 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 @@ -1,27 +1,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (586) + WholeStageCodegen (331) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (585) + WholeStageCodegen (330) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (116) + WholeStageCodegen (65) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (115) + WholeStageCodegen (64) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (38) + WholeStageCodegen (21) Filter [sales] Subquery #3 WholeStageCodegen (8) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #16 + Exchange #15 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -42,7 +42,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 + BroadcastExchange #16 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #17 + ReusedExchange [d_date_sk] #16 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -59,11 +59,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #17 + ReusedExchange [d_date_sk] #16 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (37) + WholeStageCodegen (20) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -184,22 +184,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (36) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - WholeStageCodegen (76) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (42) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #18 - WholeStageCodegen (75) + Exchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (41) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -216,14 +208,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (114) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (63) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #19 - WholeStageCodegen (113) + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (62) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -240,39 +232,39 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (233) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #20 - WholeStageCodegen (232) + Exchange [channel,i_brand_id,i_class_id] #19 + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (350) + WholeStageCodegen (197) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #21 - WholeStageCodegen (349) + Exchange [channel,i_brand_id] #20 + WholeStageCodegen (196) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (467) + WholeStageCodegen (263) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #22 - WholeStageCodegen (466) + Exchange [channel] #21 + WholeStageCodegen (262) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (584) + WholeStageCodegen (329) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #23 - WholeStageCodegen (583) + Exchange #22 + WholeStageCodegen (328) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter 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 9d2105438bbef..ed3595d8fc700 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 @@ -1,21 +1,21 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * Filter (66) - : : +- * HashAggregate (65) - : : +- Exchange (64) - : : +- * HashAggregate (63) - : : +- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) +TakeOrderedAndProject (120) ++- * HashAggregate (119) + +- Exchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- Union (92) + : :- * Filter (61) + : : +- * HashAggregate (60) + : : +- Exchange (59) + : : +- * HashAggregate (58) + : : +- * Project (57) + : : +- * BroadcastHashJoin Inner BuildRight (56) + : : :- * Project (54) + : : : +- * BroadcastHashJoin Inner BuildRight (53) : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) @@ -27,83 +27,83 @@ TakeOrderedAndProject (125) : : : : : +- * ColumnarToRow (5) : : : : : +- Scan parquet spark_catalog.default.item (4) : : : : +- BroadcastExchange (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * Filter (38) - : : : : : : +- * ColumnarToRow (37) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (36) - : : : : : +- ReusedExchange (39) - : : : : +- ReusedExchange (42) - : : : +- BroadcastExchange (57) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) - : : : :- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet spark_catalog.default.item (52) - : : : +- ReusedExchange (55) - : : +- ReusedExchange (60) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) - : : : : :- * Filter (69) - : : : : : +- * ColumnarToRow (68) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (67) - : : : : +- ReusedExchange (70) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- Exchange (94) - : +- * HashAggregate (93) - : +- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * BroadcastHashJoin LeftSemi BuildRight (86) - : : : :- * Filter (84) - : : : : +- * ColumnarToRow (83) - : : : : +- Scan parquet spark_catalog.default.web_sales (82) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) + : : : : +- * HashAggregate (46) + : : : : +- Exchange (45) + : : : : +- * HashAggregate (44) + : : : : +- * Project (43) + : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : :- * Project (40) + : : : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (38) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) + : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- BroadcastExchange (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (32) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : :- * Filter (29) + : : : : : : : +- * ColumnarToRow (28) + : : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) + : : : : : : +- ReusedExchange (30) + : : : : : +- ReusedExchange (33) + : : : : +- ReusedExchange (41) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : :- * Filter (76) + : : +- * HashAggregate (75) + : : +- Exchange (74) + : : +- * HashAggregate (73) + : : +- * Project (72) + : : +- * BroadcastHashJoin Inner BuildRight (71) + : : :- * Project (69) + : : : +- * BroadcastHashJoin Inner BuildRight (68) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (66) + : : : : :- * Filter (64) + : : : : : +- * ColumnarToRow (63) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (62) + : : : : +- ReusedExchange (65) + : : : +- ReusedExchange (67) + : : +- ReusedExchange (70) + : +- * Filter (91) + : +- * HashAggregate (90) + : +- Exchange (89) + : +- * HashAggregate (88) + : +- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * Project (84) + : : +- * BroadcastHashJoin Inner BuildRight (83) + : : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : : :- * Filter (79) + : : : : +- * ColumnarToRow (78) + : : : : +- Scan parquet spark_catalog.default.web_sales (77) + : : : +- ReusedExchange (80) + : : +- ReusedExchange (82) + : +- ReusedExchange (85) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- * HashAggregate (97) + : +- ReusedExchange (96) :- * HashAggregate (105) : +- Exchange (104) : +- * HashAggregate (103) @@ -114,16 +114,11 @@ TakeOrderedAndProject (125) : +- * HashAggregate (108) : +- * HashAggregate (107) : +- ReusedExchange (106) - :- * HashAggregate (115) - : +- Exchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- ReusedExchange (111) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- ReusedExchange (116) + +- * HashAggregate (115) + +- Exchange (114) + +- * HashAggregate (113) + +- * HashAggregate (112) + +- ReusedExchange (111) (1) Scan parquet spark_catalog.default.store_sales @@ -134,10 +129,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -163,10 +158,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 9] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -177,10 +172,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 7] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -227,7 +222,7 @@ Join condition: None Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(22) ReusedExchange [Reuses operator id: 159] +(22) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#23] (23) BroadcastHashJoin [codegen id : 3] @@ -244,205 +239,181 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 4] +(26) BroadcastHashJoin [codegen id : 7] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(27) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(28) ColumnarToRow [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(29) Filter [codegen id : 6] +Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +Condition : isnotnull(ws_item_sk#24) -(30) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [ws_item_sk#24] +Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] +Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] +Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) Exchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(33) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#30] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] +(34) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#25] +Right keys [1]: [d_date_sk#30] +Join type: Inner +Join condition: None -(36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(35) Project [codegen id : 6] +Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] -(37) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +(36) BroadcastExchange +Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] -(38) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) +(37) BroadcastHashJoin [codegen id : 7] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] +Join type: LeftSemi +Join condition: None -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(38) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] +(39) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] +(40) Project [codegen id : 9] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(42) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#34] +(41) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#31] -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] +(42) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] +(43) Project [codegen id : 9] +Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] -(45) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +(44) HashAggregate [codegen id : 9] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None +(45) Exchange +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(46) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#32, class_id#33, category_id#34] (47) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] +Input [3]: [brand_id#32, class_id#33, category_id#34] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] (50) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(52) Scan parquet spark_catalog.default.item +(52) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(53) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(54) Filter [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#35] - -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None -(57) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(58) BroadcastHashJoin [codegen id : 25] +(53) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 25] +(54) Project [codegen id : 14] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(60) ReusedExchange [Reuses operator id: 154] +(55) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#40] -(61) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(62) Project [codegen id : 25] +(57) Project [codegen id : 14] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] -(63) HashAggregate [codegen id : 25] +(58) HashAggregate [codegen id : 14] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -(64) Exchange +(59) Exchange Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(65) HashAggregate [codegen id : 26] +(60) HashAggregate [codegen id : 15] Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(66) Filter [codegen id : 26] +(61) Filter [codegen id : 15] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(67) Scan parquet spark_catalog.default.catalog_sales +(62) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -450,71 +421,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 51] +(63) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(69) Filter [codegen id : 51] +(64) Filter [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Condition : isnotnull(cs_item_sk#54) -(70) ReusedExchange [Reuses operator id: 50] +(65) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#35] -(71) BroadcastHashJoin [codegen id : 51] +(66) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(72) ReusedExchange [Reuses operator id: 57] +(67) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] -(73) BroadcastHashJoin [codegen id : 51] +(68) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [i_item_sk#58] Join type: Inner Join condition: None -(74) Project [codegen id : 51] +(69) Project [codegen id : 29] Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59, i_class_id#60, i_category_id#61] Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] -(75) ReusedExchange [Reuses operator id: 154] +(70) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#62] -(76) BroadcastHashJoin [codegen id : 51] +(71) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#57] Right keys [1]: [d_date_sk#62] Join type: Inner Join condition: None -(77) Project [codegen id : 51] +(72) Project [codegen id : 29] Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61] Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59, i_class_id#60, i_category_id#61, d_date_sk#62] -(78) HashAggregate [codegen id : 51] +(73) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61] Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61] Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] Results [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] -(79) Exchange +(74) Exchange Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(80) HashAggregate [codegen id : 52] +(75) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61] Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70] Results [6]: [catalog AS channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73] -(81) Filter [codegen id : 52] +(76) Filter [codegen id : 30] Input [6]: [channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sales#72, number_sales#73] Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(82) Scan parquet spark_catalog.default.web_sales +(77) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] @@ -522,443 +493,443 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 77] +(78) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -(84) Filter [codegen id : 77] +(79) Filter [codegen id : 44] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Condition : isnotnull(ws_item_sk#74) -(85) ReusedExchange [Reuses operator id: 50] +(80) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#35] -(86) BroadcastHashJoin [codegen id : 77] +(81) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#74] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(87) ReusedExchange [Reuses operator id: 57] +(82) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -(88) BroadcastHashJoin [codegen id : 77] +(83) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#74] Right keys [1]: [i_item_sk#78] Join type: Inner Join condition: None -(89) Project [codegen id : 77] +(84) Project [codegen id : 44] Output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79, i_class_id#80, i_category_id#81] Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -(90) ReusedExchange [Reuses operator id: 154] +(85) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#82] -(91) BroadcastHashJoin [codegen id : 77] +(86) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#77] Right keys [1]: [d_date_sk#82] Join type: Inner Join condition: None -(92) Project [codegen id : 77] +(87) Project [codegen id : 44] Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81] Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] -(93) HashAggregate [codegen id : 77] +(88) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81] Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -(94) Exchange +(89) Exchange Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(95) HashAggregate [codegen id : 78] +(90) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90] Results [6]: [web AS channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93] -(96) Filter [codegen id : 78] +(91) Filter [codegen id : 45] Input [6]: [channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sales#92, number_sales#93] Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(97) Union +(92) Union -(98) HashAggregate [codegen id : 79] +(93) HashAggregate [codegen id : 46] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] Aggregate Attributes [3]: [sum#94, isEmpty#95, sum#96] Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(99) Exchange +(94) Exchange Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(100) HashAggregate [codegen id : 80] +(95) HashAggregate [codegen id : 47] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(101) ReusedExchange [Reuses operator id: 99] +(96) ReusedExchange [Reuses operator id: 94] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(102) HashAggregate [codegen id : 160] +(97) HashAggregate [codegen id : 94] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [5]: [channel#49, i_brand_id#37, i_class_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(103) HashAggregate [codegen id : 160] +(98) HashAggregate [codegen id : 94] Input [5]: [channel#49, i_brand_id#37, i_class_id#38, sum_sales#102, number_sales#103] Keys [3]: [channel#49, i_brand_id#37, i_class_id#38] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] -(104) Exchange +(99) Exchange Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(105) HashAggregate [codegen id : 161] +(100) HashAggregate [codegen id : 95] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] Keys [3]: [channel#49, i_brand_id#37, i_class_id#38] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#110, sum(number_sales#103)#111] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, null AS i_category_id#112, sum(sum_sales#102)#110 AS sum(sum_sales)#113, sum(number_sales#103)#111 AS sum(number_sales)#114] -(106) ReusedExchange [Reuses operator id: 99] +(101) ReusedExchange [Reuses operator id: 94] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(107) HashAggregate [codegen id : 241] +(102) HashAggregate [codegen id : 142] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [4]: [channel#49, i_brand_id#37, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(108) HashAggregate [codegen id : 241] +(103) HashAggregate [codegen id : 142] Input [4]: [channel#49, i_brand_id#37, sum_sales#102, number_sales#103] Keys [2]: [channel#49, i_brand_id#37] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] Results [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] -(109) Exchange +(104) Exchange Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#49, i_brand_id#37, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(channel#49, i_brand_id#37, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(110) HashAggregate [codegen id : 242] +(105) HashAggregate [codegen id : 143] Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] Keys [2]: [channel#49, i_brand_id#37] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#121, sum(number_sales#103)#122] Results [6]: [channel#49, i_brand_id#37, null AS i_class_id#123, null AS i_category_id#124, sum(sum_sales#102)#121 AS sum(sum_sales)#125, sum(number_sales#103)#122 AS sum(number_sales)#126] -(111) ReusedExchange [Reuses operator id: 99] +(106) ReusedExchange [Reuses operator id: 94] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(112) HashAggregate [codegen id : 322] +(107) HashAggregate [codegen id : 190] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(113) HashAggregate [codegen id : 322] +(108) HashAggregate [codegen id : 190] Input [3]: [channel#49, sum_sales#102, number_sales#103] Keys [1]: [channel#49] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129] Results [4]: [channel#49, sum#130, isEmpty#131, sum#132] -(114) Exchange +(109) Exchange Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(115) HashAggregate [codegen id : 323] +(110) HashAggregate [codegen id : 191] Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] Keys [1]: [channel#49] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#133, sum(number_sales#103)#134] Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null AS i_category_id#137, sum(sum_sales#102)#133 AS sum(sum_sales)#138, sum(number_sales#103)#134 AS sum(number_sales)#139] -(116) ReusedExchange [Reuses operator id: 99] +(111) ReusedExchange [Reuses operator id: 94] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(117) HashAggregate [codegen id : 403] +(112) HashAggregate [codegen id : 238] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(118) HashAggregate [codegen id : 403] +(113) HashAggregate [codegen id : 238] Input [2]: [sum_sales#102, number_sales#103] Keys: [] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#140, isEmpty#141, sum#142] Results [3]: [sum#143, isEmpty#144, sum#145] -(119) Exchange +(114) Exchange Input [3]: [sum#143, isEmpty#144, sum#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(120) HashAggregate [codegen id : 404] +(115) HashAggregate [codegen id : 239] Input [3]: [sum#143, isEmpty#144, sum#145] Keys: [] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#146, sum(number_sales#103)#147] Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#150, null AS i_category_id#151, sum(sum_sales#102)#146 AS sum(sum_sales)#152, sum(number_sales#103)#147 AS sum(number_sales)#153] -(121) Union +(116) Union -(122) HashAggregate [codegen id : 405] +(117) HashAggregate [codegen id : 240] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -(123) Exchange +(118) Exchange Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(124) HashAggregate [codegen id : 406] +(119) HashAggregate [codegen id : 241] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -(125) TakeOrderedAndProject +(120) TakeOrderedAndProject Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (144) -+- Exchange (143) - +- * HashAggregate (142) - +- Union (141) +Subquery:1 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (139) ++- Exchange (138) + +- * HashAggregate (137) + +- Union (136) + :- * Project (125) + : +- * BroadcastHashJoin Inner BuildRight (124) + : :- * ColumnarToRow (122) + : : +- Scan parquet spark_catalog.default.store_sales (121) + : +- ReusedExchange (123) :- * Project (130) : +- * BroadcastHashJoin Inner BuildRight (129) : :- * ColumnarToRow (127) - : : +- Scan parquet spark_catalog.default.store_sales (126) + : : +- Scan parquet spark_catalog.default.catalog_sales (126) : +- ReusedExchange (128) - :- * Project (135) - : +- * BroadcastHashJoin Inner BuildRight (134) - : :- * ColumnarToRow (132) - : : +- Scan parquet spark_catalog.default.catalog_sales (131) - : +- ReusedExchange (133) - +- * Project (140) - +- * BroadcastHashJoin Inner BuildRight (139) - :- * ColumnarToRow (137) - : +- Scan parquet spark_catalog.default.web_sales (136) - +- ReusedExchange (138) - - -(126) Scan parquet spark_catalog.default.store_sales + +- * Project (135) + +- * BroadcastHashJoin Inner BuildRight (134) + :- * ColumnarToRow (132) + : +- Scan parquet spark_catalog.default.web_sales (131) + +- ReusedExchange (133) + + +(121) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 2] +(122) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] -(128) ReusedExchange [Reuses operator id: 159] +(123) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#157] -(129) BroadcastHashJoin [codegen id : 2] +(124) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#156] Right keys [1]: [d_date_sk#157] Join type: Inner Join condition: None -(130) Project [codegen id : 2] +(125) Project [codegen id : 2] Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159] Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157] -(131) Scan parquet spark_catalog.default.catalog_sales +(126) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 4] +(127) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] -(133) ReusedExchange [Reuses operator id: 149] +(128) ReusedExchange [Reuses operator id: 144] Output [1]: [d_date_sk#164] -(134) BroadcastHashJoin [codegen id : 4] +(129) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#162] Right keys [1]: [d_date_sk#164] Join type: Inner Join condition: None -(135) Project [codegen id : 4] +(130) Project [codegen id : 4] Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166] Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164] -(136) Scan parquet spark_catalog.default.web_sales +(131) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 6] +(132) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] -(138) ReusedExchange [Reuses operator id: 149] +(133) ReusedExchange [Reuses operator id: 144] Output [1]: [d_date_sk#170] -(139) BroadcastHashJoin [codegen id : 6] +(134) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#169] Right keys [1]: [d_date_sk#170] Join type: Inner Join condition: None -(140) Project [codegen id : 6] +(135) Project [codegen id : 6] Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172] Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170] -(141) Union +(136) Union -(142) HashAggregate [codegen id : 7] +(137) HashAggregate [codegen id : 7] Input [2]: [quantity#158, list_price#159] Keys: [] Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] Aggregate Attributes [2]: [sum#173, count#174] Results [2]: [sum#175, count#176] -(143) Exchange +(138) Exchange Input [2]: [sum#175, count#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] -(144) HashAggregate [codegen id : 8] +(139) HashAggregate [codegen id : 8] Input [2]: [sum#175, count#176] Keys: [] Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177] Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 121 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 -BroadcastExchange (149) -+- * Project (148) - +- * Filter (147) - +- * ColumnarToRow (146) - +- Scan parquet spark_catalog.default.date_dim (145) +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 +BroadcastExchange (144) ++- * Project (143) + +- * Filter (142) + +- * ColumnarToRow (141) + +- Scan parquet spark_catalog.default.date_dim (140) -(145) Scan parquet spark_catalog.default.date_dim +(140) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#164, d_year#179] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(146) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#164, d_year#179] -(147) Filter [codegen id : 1] +(142) Filter [codegen id : 1] Input [2]: [d_date_sk#164, d_year#179] Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#164)) -(148) Project [codegen id : 1] +(143) Project [codegen id : 1] Output [1]: [d_date_sk#164] Input [2]: [d_date_sk#164, d_year#179] -(149) BroadcastExchange +(144) BroadcastExchange Input [1]: [d_date_sk#164] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] -Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (154) -+- * Project (153) - +- * Filter (152) - +- * ColumnarToRow (151) - +- Scan parquet spark_catalog.default.date_dim (150) +BroadcastExchange (149) ++- * Project (148) + +- * Filter (147) + +- * ColumnarToRow (146) + +- Scan parquet spark_catalog.default.date_dim (145) -(150) Scan parquet spark_catalog.default.date_dim +(145) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#40, d_year#180, d_moy#181] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 1] +(146) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#40, d_year#180, d_moy#181] -(152) Filter [codegen id : 1] +(147) Filter [codegen id : 1] Input [3]: [d_date_sk#40, d_year#180, d_moy#181] Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 = 2000)) AND (d_moy#181 = 11)) AND isnotnull(d_date_sk#40)) -(153) Project [codegen id : 1] +(148) Project [codegen id : 1] Output [1]: [d_date_sk#40] Input [3]: [d_date_sk#40, d_year#180, d_moy#181] -(154) BroadcastExchange +(149) BroadcastExchange Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (159) -+- * Project (158) - +- * Filter (157) - +- * ColumnarToRow (156) - +- Scan parquet spark_catalog.default.date_dim (155) +BroadcastExchange (154) ++- * Project (153) + +- * Filter (152) + +- * ColumnarToRow (151) + +- Scan parquet spark_catalog.default.date_dim (150) -(155) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#182] +(150) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_year#182] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(156) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#182] +(151) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#182] -(157) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#182] -Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#24)) +(152) Filter [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#182] +Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#31)) -(158) Project [codegen id : 1] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#182] +(153) Project [codegen id : 1] +Output [1]: [d_date_sk#31] +Input [2]: [d_date_sk#31, d_year#182] -(159) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +(154) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 76 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 62 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:11 Hosting operator id = 91 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 77 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 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 ecd7e897d97b2..1c4835db87cf1 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 @@ -1,27 +1,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (406) + WholeStageCodegen (241) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) + WholeStageCodegen (240) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (80) + WholeStageCodegen (47) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) + WholeStageCodegen (46) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (15) Filter [sales] Subquery #3 WholeStageCodegen (8) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #14 + Exchange #13 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -42,7 +42,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 + BroadcastExchange #14 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #14 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -59,11 +59,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #14 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -94,31 +94,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #6 WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - 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 - BroadcastExchange #9 - WholeStageCodegen (4) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + 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 + BroadcastExchange #9 + WholeStageCodegen (7) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -145,43 +145,35 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) + WholeStageCodegen (30) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (29) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -196,16 +188,16 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) + WholeStageCodegen (45) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -220,41 +212,41 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) + WholeStageCodegen (95) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) + Exchange [channel,i_brand_id,i_class_id] #17 + WholeStageCodegen (94) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) + WholeStageCodegen (143) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #19 - WholeStageCodegen (241) + Exchange [channel,i_brand_id] #18 + WholeStageCodegen (142) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) + WholeStageCodegen (191) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #20 - WholeStageCodegen (322) + Exchange [channel] #19 + WholeStageCodegen (190) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) + WholeStageCodegen (239) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #21 - WholeStageCodegen (403) + Exchange #20 + WholeStageCodegen (238) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt index 2671e37f4e1ef..6203d0589a2f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt @@ -1,160 +1,164 @@ == Physical Plan == -TakeOrderedAndProject (156) -+- Union (155) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet spark_catalog.default.item (14) - : +- * Sort (42) - : +- Exchange (41) - : +- * Project (40) - : +- * SortMergeJoin Inner (39) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (25) - : : : +- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet spark_catalog.default.customer (22) - : : +- BroadcastExchange (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet spark_catalog.default.customer_address (26) - : +- * Sort (38) - : +- Exchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet spark_catalog.default.customer_demographics (34) - :- * HashAggregate (72) - : +- Exchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * SortMergeJoin Inner (68) - : :- * Sort (49) - : : +- ReusedExchange (48) - : +- * Sort (67) - : +- Exchange (66) - : +- * Project (65) - : +- * SortMergeJoin Inner (64) - : :- * Sort (61) - : : +- Exchange (60) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (53) - : : : +- * Filter (52) - : : : +- * ColumnarToRow (51) - : : : +- Scan parquet spark_catalog.default.customer (50) - : : +- BroadcastExchange (57) - : : +- * Filter (56) - : : +- * ColumnarToRow (55) - : : +- Scan parquet spark_catalog.default.customer_address (54) - : +- * Sort (63) - : +- ReusedExchange (62) - :- * HashAggregate (98) - : +- Exchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * SortMergeJoin Inner (94) - : :- * Sort (74) - : : +- ReusedExchange (73) - : +- * Sort (93) - : +- Exchange (92) - : +- * Project (91) - : +- * SortMergeJoin Inner (90) - : :- * Sort (87) - : : +- Exchange (86) - : : +- * Project (85) - : : +- * BroadcastHashJoin Inner BuildRight (84) - : : :- * Project (78) - : : : +- * Filter (77) - : : : +- * ColumnarToRow (76) - : : : +- Scan parquet spark_catalog.default.customer (75) - : : +- BroadcastExchange (83) - : : +- * Project (82) - : : +- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet spark_catalog.default.customer_address (79) - : +- * Sort (89) - : +- ReusedExchange (88) - :- * HashAggregate (133) - : +- Exchange (132) - : +- * HashAggregate (131) - : +- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * Project (127) - : : +- * BroadcastHashJoin Inner BuildRight (126) - : : :- * Project (107) - : : : +- * BroadcastHashJoin Inner BuildRight (106) - : : : :- * Project (104) - : : : : +- * BroadcastHashJoin Inner BuildRight (103) - : : : : :- * Filter (101) - : : : : : +- * ColumnarToRow (100) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (99) - : : : : +- ReusedExchange (102) - : : : +- ReusedExchange (105) - : : +- BroadcastExchange (125) - : : +- * Project (124) - : : +- * BroadcastHashJoin Inner BuildLeft (123) - : : :- BroadcastExchange (119) - : : : +- * Project (118) - : : : +- * BroadcastHashJoin Inner BuildRight (117) - : : : :- * Project (111) - : : : : +- * Filter (110) - : : : : +- * ColumnarToRow (109) - : : : : +- Scan parquet spark_catalog.default.customer (108) - : : : +- BroadcastExchange (116) - : : : +- * Project (115) - : : : +- * Filter (114) - : : : +- * ColumnarToRow (113) - : : : +- Scan parquet spark_catalog.default.customer_address (112) - : : +- * Filter (122) - : : +- * ColumnarToRow (121) - : : +- Scan parquet spark_catalog.default.customer_demographics (120) - : +- ReusedExchange (128) - +- * HashAggregate (154) - +- Exchange (153) - +- * HashAggregate (152) - +- * Project (151) - +- * BroadcastHashJoin Inner BuildRight (150) - :- * Project (148) - : +- * BroadcastHashJoin Inner BuildRight (147) - : :- * Project (142) - : : +- * BroadcastHashJoin Inner BuildRight (141) - : : :- * Project (139) - : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : :- * Filter (136) - : : : : +- * ColumnarToRow (135) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (134) - : : : +- ReusedExchange (137) - : : +- ReusedExchange (140) - : +- BroadcastExchange (146) - : +- * Filter (145) - : +- * ColumnarToRow (144) - : +- Scan parquet spark_catalog.default.item (143) - +- ReusedExchange (149) +TakeOrderedAndProject (160) ++- Union (159) + :- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * Project (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet spark_catalog.default.item (14) + : +- * Sort (42) + : +- Exchange (41) + : +- * Project (40) + : +- * SortMergeJoin Inner (39) + : :- * Sort (33) + : : +- Exchange (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (25) + : : : +- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet spark_catalog.default.customer (22) + : : +- BroadcastExchange (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet spark_catalog.default.customer_address (26) + : +- * Sort (38) + : +- Exchange (37) + : +- * Filter (36) + : +- * ColumnarToRow (35) + : +- Scan parquet spark_catalog.default.customer_demographics (34) + :- * HashAggregate (74) + : +- Exchange (73) + : +- * HashAggregate (72) + : +- * Project (71) + : +- * Project (70) + : +- * SortMergeJoin Inner (69) + : :- * Sort (50) + : : +- ReusedExchange (49) + : +- * Sort (68) + : +- Exchange (67) + : +- * Project (66) + : +- * SortMergeJoin Inner (65) + : :- * Sort (62) + : : +- Exchange (61) + : : +- * Project (60) + : : +- * BroadcastHashJoin Inner BuildRight (59) + : : :- * Project (54) + : : : +- * Filter (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet spark_catalog.default.customer (51) + : : +- BroadcastExchange (58) + : : +- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet spark_catalog.default.customer_address (55) + : +- * Sort (64) + : +- ReusedExchange (63) + :- * HashAggregate (101) + : +- Exchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * Project (97) + : +- * SortMergeJoin Inner (96) + : :- * Sort (76) + : : +- ReusedExchange (75) + : +- * Sort (95) + : +- Exchange (94) + : +- * Project (93) + : +- * SortMergeJoin Inner (92) + : :- * Sort (89) + : : +- Exchange (88) + : : +- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * Project (80) + : : : +- * Filter (79) + : : : +- * ColumnarToRow (78) + : : : +- Scan parquet spark_catalog.default.customer (77) + : : +- BroadcastExchange (85) + : : +- * Project (84) + : : +- * Filter (83) + : : +- * ColumnarToRow (82) + : : +- Scan parquet spark_catalog.default.customer_address (81) + : +- * Sort (91) + : +- ReusedExchange (90) + :- * HashAggregate (136) + : +- Exchange (135) + : +- * HashAggregate (134) + : +- * Project (133) + : +- * BroadcastHashJoin Inner BuildRight (132) + : :- * Project (130) + : : +- * BroadcastHashJoin Inner BuildRight (129) + : : :- * Project (110) + : : : +- * BroadcastHashJoin Inner BuildRight (109) + : : : :- * Project (107) + : : : : +- * BroadcastHashJoin Inner BuildRight (106) + : : : : :- * Filter (104) + : : : : : +- * ColumnarToRow (103) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (102) + : : : : +- ReusedExchange (105) + : : : +- ReusedExchange (108) + : : +- BroadcastExchange (128) + : : +- * Project (127) + : : +- * BroadcastHashJoin Inner BuildLeft (126) + : : :- BroadcastExchange (122) + : : : +- * Project (121) + : : : +- * BroadcastHashJoin Inner BuildRight (120) + : : : :- * Project (114) + : : : : +- * Filter (113) + : : : : +- * ColumnarToRow (112) + : : : : +- Scan parquet spark_catalog.default.customer (111) + : : : +- BroadcastExchange (119) + : : : +- * Project (118) + : : : +- * Filter (117) + : : : +- * ColumnarToRow (116) + : : : +- Scan parquet spark_catalog.default.customer_address (115) + : : +- * Filter (125) + : : +- * ColumnarToRow (124) + : : +- Scan parquet spark_catalog.default.customer_demographics (123) + : +- ReusedExchange (131) + +- * HashAggregate (158) + +- Exchange (157) + +- * HashAggregate (156) + +- * Project (155) + +- * Project (154) + +- * BroadcastHashJoin Inner BuildRight (153) + :- * Project (151) + : +- * BroadcastHashJoin Inner BuildRight (150) + : :- * Project (145) + : : +- * BroadcastHashJoin Inner BuildRight (144) + : : :- * Project (142) + : : : +- * BroadcastHashJoin Inner BuildRight (141) + : : : :- * Filter (139) + : : : : +- * ColumnarToRow (138) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (137) + : : : +- ReusedExchange (140) + : : +- ReusedExchange (143) + : +- BroadcastExchange (149) + : +- * Filter (148) + : +- * ColumnarToRow (147) + : +- Scan parquet spark_catalog.default.item (146) + +- ReusedExchange (152) (1) Scan parquet spark_catalog.default.catalog_sales @@ -204,7 +208,7 @@ Join condition: None Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(11) ReusedExchange [Reuses operator id: 161] +(11) ReusedExchange [Reuses operator id: 165] Output [1]: [d_date_sk#15] (12) BroadcastHashJoin [codegen id : 4] @@ -354,260 +358,272 @@ Join type: Inner Join condition: None (44) Project [codegen id : 13] -Output [11]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, ca_county#24, ca_state#25, ca_country#26, i_item_id#17] Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17, c_customer_sk#18, c_birth_year#22, ca_county#24, ca_state#25, ca_country#26] -(45) HashAggregate [codegen id : 13] +(45) Project [codegen id : 13] +Output [11]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, ca_county#24, ca_state#25, ca_country#26, i_item_id#17] + +(46) HashAggregate [codegen id : 13] Input [11]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [4]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] Results [18]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -(46) Exchange +(47) Exchange Input [18]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Arguments: hashpartitioning(i_item_id#17, ca_country#26, ca_state#25, ca_county#24, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [18]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [4]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] Aggregate Attributes [7]: [avg(agg1#28)#63, avg(agg2#29)#64, avg(agg3#30)#65, avg(agg4#31)#66, avg(agg5#32)#67, avg(agg6#33)#68, avg(agg7#34)#69] Results [11]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, avg(agg1#28)#63 AS agg1#70, avg(agg2#29)#64 AS agg2#71, avg(agg3#30)#65 AS agg3#72, avg(agg4#31)#66 AS agg4#73, avg(agg5#32)#67 AS agg5#74, avg(agg6#33)#68 AS agg6#75, avg(agg7#34)#69 AS agg7#76] -(48) ReusedExchange [Reuses operator id: 20] +(49) ReusedExchange [Reuses operator id: 20] Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17] -(49) Sort [codegen id : 19] +(50) Sort [codegen id : 19] Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(50) Scan parquet spark_catalog.default.customer +(51) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 21] +(52) ColumnarToRow [codegen id : 21] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] -(52) Filter [codegen id : 21] +(53) Filter [codegen id : 21] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] Condition : (((c_birth_month#21 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#18)) AND isnotnull(c_current_cdemo_sk#19)) AND isnotnull(c_current_addr_sk#20)) -(53) Project [codegen id : 21] +(54) Project [codegen id : 21] Output [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_year#22] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] -(54) Scan parquet spark_catalog.default.customer_address +(55) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#23, ca_state#25, ca_country#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 20] +(56) ColumnarToRow [codegen id : 20] Input [3]: [ca_address_sk#23, ca_state#25, ca_country#26] -(56) Filter [codegen id : 20] +(57) Filter [codegen id : 20] Input [3]: [ca_address_sk#23, ca_state#25, ca_country#26] Condition : (ca_state#25 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#23)) -(57) BroadcastExchange +(58) BroadcastExchange Input [3]: [ca_address_sk#23, ca_state#25, ca_country#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(58) BroadcastHashJoin [codegen id : 21] +(59) BroadcastHashJoin [codegen id : 21] Left keys [1]: [c_current_addr_sk#20] Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(59) Project [codegen id : 21] +(60) Project [codegen id : 21] Output [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_state#25, ca_country#26] Input [7]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_year#22, ca_address_sk#23, ca_state#25, ca_country#26] -(60) Exchange +(61) Exchange Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_state#25, ca_country#26] Arguments: hashpartitioning(c_current_cdemo_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(61) Sort [codegen id : 22] +(62) Sort [codegen id : 22] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_state#25, ca_country#26] Arguments: [c_current_cdemo_sk#19 ASC NULLS FIRST], false, 0 -(62) ReusedExchange [Reuses operator id: 37] +(63) ReusedExchange [Reuses operator id: 37] Output [1]: [cd_demo_sk#27] -(63) Sort [codegen id : 24] +(64) Sort [codegen id : 24] Input [1]: [cd_demo_sk#27] Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin [codegen id : 25] +(65) SortMergeJoin [codegen id : 25] Left keys [1]: [c_current_cdemo_sk#19] Right keys [1]: [cd_demo_sk#27] Join type: Inner Join condition: None -(65) Project [codegen id : 25] +(66) Project [codegen id : 25] Output [4]: [c_customer_sk#18, c_birth_year#22, ca_state#25, ca_country#26] Input [6]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_state#25, ca_country#26, cd_demo_sk#27] -(66) Exchange +(67) Exchange Input [4]: [c_customer_sk#18, c_birth_year#22, ca_state#25, ca_country#26] Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(67) Sort [codegen id : 26] +(68) Sort [codegen id : 26] Input [4]: [c_customer_sk#18, c_birth_year#22, ca_state#25, ca_country#26] Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin [codegen id : 27] +(69) SortMergeJoin [codegen id : 27] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(69) Project [codegen id : 27] -Output [10]: [i_item_id#17, ca_country#26, ca_state#25, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +(70) Project [codegen id : 27] +Output [10]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, ca_state#25, ca_country#26, i_item_id#17] Input [12]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17, c_customer_sk#18, c_birth_year#22, ca_state#25, ca_country#26] -(70) HashAggregate [codegen id : 27] +(71) Project [codegen id : 27] +Output [10]: [i_item_id#17, ca_country#26, ca_state#25, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +Input [10]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, ca_state#25, ca_country#26, i_item_id#17] + +(72) HashAggregate [codegen id : 27] Input [10]: [i_item_id#17, ca_country#26, ca_state#25, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [3]: [i_item_id#17, ca_country#26, ca_state#25] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] Results [17]: [i_item_id#17, ca_country#26, ca_state#25, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] -(71) Exchange +(73) Exchange Input [17]: [i_item_id#17, ca_country#26, ca_state#25, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] Arguments: hashpartitioning(i_item_id#17, ca_country#26, ca_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(72) HashAggregate [codegen id : 28] +(74) HashAggregate [codegen id : 28] Input [17]: [i_item_id#17, ca_country#26, ca_state#25, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] Keys [3]: [i_item_id#17, ca_country#26, ca_state#25] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] Aggregate Attributes [7]: [avg(agg1#28)#105, avg(agg2#29)#106, avg(agg3#30)#107, avg(agg4#31)#108, avg(agg5#32)#109, avg(agg6#33)#110, avg(agg7#34)#111] Results [11]: [i_item_id#17, ca_country#26, ca_state#25, null AS county#112, avg(agg1#28)#105 AS agg1#113, avg(agg2#29)#106 AS agg2#114, avg(agg3#30)#107 AS agg3#115, avg(agg4#31)#108 AS agg4#116, avg(agg5#32)#109 AS agg5#117, avg(agg6#33)#110 AS agg6#118, avg(agg7#34)#111 AS agg7#119] -(73) ReusedExchange [Reuses operator id: 20] +(75) ReusedExchange [Reuses operator id: 20] Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17] -(74) Sort [codegen id : 33] +(76) Sort [codegen id : 33] Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(75) Scan parquet spark_catalog.default.customer +(77) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 35] +(78) ColumnarToRow [codegen id : 35] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] -(77) Filter [codegen id : 35] +(79) Filter [codegen id : 35] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] Condition : (((c_birth_month#21 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#18)) AND isnotnull(c_current_cdemo_sk#19)) AND isnotnull(c_current_addr_sk#20)) -(78) Project [codegen id : 35] +(80) Project [codegen id : 35] Output [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_year#22] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] -(79) Scan parquet spark_catalog.default.customer_address +(81) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#23, ca_state#25, ca_country#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 34] +(82) ColumnarToRow [codegen id : 34] Input [3]: [ca_address_sk#23, ca_state#25, ca_country#26] -(81) Filter [codegen id : 34] +(83) Filter [codegen id : 34] Input [3]: [ca_address_sk#23, ca_state#25, ca_country#26] Condition : (ca_state#25 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#23)) -(82) Project [codegen id : 34] +(84) Project [codegen id : 34] Output [2]: [ca_address_sk#23, ca_country#26] Input [3]: [ca_address_sk#23, ca_state#25, ca_country#26] -(83) BroadcastExchange +(85) BroadcastExchange Input [2]: [ca_address_sk#23, ca_country#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(84) BroadcastHashJoin [codegen id : 35] +(86) BroadcastHashJoin [codegen id : 35] Left keys [1]: [c_current_addr_sk#20] Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(85) Project [codegen id : 35] +(87) Project [codegen id : 35] Output [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_country#26] Input [6]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_year#22, ca_address_sk#23, ca_country#26] -(86) Exchange +(88) Exchange Input [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_country#26] Arguments: hashpartitioning(c_current_cdemo_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(87) Sort [codegen id : 36] +(89) Sort [codegen id : 36] Input [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_country#26] Arguments: [c_current_cdemo_sk#19 ASC NULLS FIRST], false, 0 -(88) ReusedExchange [Reuses operator id: 37] +(90) ReusedExchange [Reuses operator id: 37] Output [1]: [cd_demo_sk#27] -(89) Sort [codegen id : 38] +(91) Sort [codegen id : 38] Input [1]: [cd_demo_sk#27] Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 39] +(92) SortMergeJoin [codegen id : 39] Left keys [1]: [c_current_cdemo_sk#19] Right keys [1]: [cd_demo_sk#27] Join type: Inner Join condition: None -(91) Project [codegen id : 39] +(93) Project [codegen id : 39] Output [3]: [c_customer_sk#18, c_birth_year#22, ca_country#26] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, ca_country#26, cd_demo_sk#27] -(92) Exchange +(94) Exchange Input [3]: [c_customer_sk#18, c_birth_year#22, ca_country#26] Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(93) Sort [codegen id : 40] +(95) Sort [codegen id : 40] Input [3]: [c_customer_sk#18, c_birth_year#22, ca_country#26] Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 -(94) SortMergeJoin [codegen id : 41] +(96) SortMergeJoin [codegen id : 41] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(95) Project [codegen id : 41] -Output [9]: [i_item_id#17, ca_country#26, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +(97) Project [codegen id : 41] +Output [9]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, ca_country#26, i_item_id#17] Input [11]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#17, c_customer_sk#18, c_birth_year#22, ca_country#26] -(96) HashAggregate [codegen id : 41] +(98) Project [codegen id : 41] +Output [9]: [i_item_id#17, ca_country#26, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +Input [9]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, ca_country#26, i_item_id#17] + +(99) HashAggregate [codegen id : 41] Input [9]: [i_item_id#17, ca_country#26, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [2]: [i_item_id#17, ca_country#26] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133] Results [16]: [i_item_id#17, ca_country#26, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147] -(97) Exchange +(100) Exchange Input [16]: [i_item_id#17, ca_country#26, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147] Arguments: hashpartitioning(i_item_id#17, ca_country#26, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(98) HashAggregate [codegen id : 42] +(101) HashAggregate [codegen id : 42] Input [16]: [i_item_id#17, ca_country#26, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147] Keys [2]: [i_item_id#17, ca_country#26] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] Aggregate Attributes [7]: [avg(agg1#28)#148, avg(agg2#29)#149, avg(agg3#30)#150, avg(agg4#31)#151, avg(agg5#32)#152, avg(agg6#33)#153, avg(agg7#34)#154] Results [11]: [i_item_id#17, ca_country#26, null AS ca_state#155, null AS county#156, avg(agg1#28)#148 AS agg1#157, avg(agg2#29)#149 AS agg2#158, avg(agg3#30)#150 AS agg3#159, avg(agg4#31)#151 AS agg4#160, avg(agg5#32)#152 AS agg5#161, avg(agg6#33)#153 AS agg6#162, avg(agg7#34)#154 AS agg7#163] -(99) Scan parquet spark_catalog.default.catalog_sales +(102) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -615,163 +631,163 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 49] +(103) ColumnarToRow [codegen id : 49] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(101) Filter [codegen id : 49] +(104) Filter [codegen id : 49] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(102) ReusedExchange [Reuses operator id: 8] +(105) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(103) BroadcastHashJoin [codegen id : 49] +(106) BroadcastHashJoin [codegen id : 49] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(104) Project [codegen id : 49] +(107) Project [codegen id : 49] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(105) ReusedExchange [Reuses operator id: 161] +(108) ReusedExchange [Reuses operator id: 165] Output [1]: [d_date_sk#15] -(106) BroadcastHashJoin [codegen id : 49] +(109) BroadcastHashJoin [codegen id : 49] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(107) Project [codegen id : 49] +(110) Project [codegen id : 49] Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#15] -(108) Scan parquet spark_catalog.default.customer +(111) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(109) ColumnarToRow [codegen id : 46] +(112) ColumnarToRow [codegen id : 46] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] -(110) Filter [codegen id : 46] +(113) Filter [codegen id : 46] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] Condition : (((c_birth_month#21 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#18)) AND isnotnull(c_current_cdemo_sk#19)) AND isnotnull(c_current_addr_sk#20)) -(111) Project [codegen id : 46] +(114) Project [codegen id : 46] Output [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_year#22] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_month#21, c_birth_year#22] -(112) Scan parquet spark_catalog.default.customer_address +(115) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#23, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(113) ColumnarToRow [codegen id : 45] +(116) ColumnarToRow [codegen id : 45] Input [2]: [ca_address_sk#23, ca_state#25] -(114) Filter [codegen id : 45] +(117) Filter [codegen id : 45] Input [2]: [ca_address_sk#23, ca_state#25] Condition : (ca_state#25 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#23)) -(115) Project [codegen id : 45] +(118) Project [codegen id : 45] Output [1]: [ca_address_sk#23] Input [2]: [ca_address_sk#23, ca_state#25] -(116) BroadcastExchange +(119) BroadcastExchange Input [1]: [ca_address_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -(117) BroadcastHashJoin [codegen id : 46] +(120) BroadcastHashJoin [codegen id : 46] Left keys [1]: [c_current_addr_sk#20] Right keys [1]: [ca_address_sk#23] Join type: Inner Join condition: None -(118) Project [codegen id : 46] +(121) Project [codegen id : 46] Output [3]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22] Input [5]: [c_customer_sk#18, c_current_cdemo_sk#19, c_current_addr_sk#20, c_birth_year#22, ca_address_sk#23] -(119) BroadcastExchange +(122) BroadcastExchange Input [3]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=18] -(120) Scan parquet spark_catalog.default.customer_demographics +(123) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(121) ColumnarToRow +(124) ColumnarToRow Input [1]: [cd_demo_sk#27] -(122) Filter +(125) Filter Input [1]: [cd_demo_sk#27] Condition : isnotnull(cd_demo_sk#27) -(123) BroadcastHashJoin [codegen id : 47] +(126) BroadcastHashJoin [codegen id : 47] Left keys [1]: [c_current_cdemo_sk#19] Right keys [1]: [cd_demo_sk#27] Join type: Inner Join condition: None -(124) Project [codegen id : 47] +(127) Project [codegen id : 47] Output [2]: [c_customer_sk#18, c_birth_year#22] Input [4]: [c_customer_sk#18, c_current_cdemo_sk#19, c_birth_year#22, cd_demo_sk#27] -(125) BroadcastExchange +(128) BroadcastExchange Input [2]: [c_customer_sk#18, c_birth_year#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -(126) BroadcastHashJoin [codegen id : 49] +(129) BroadcastHashJoin [codegen id : 49] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(127) Project [codegen id : 49] +(130) Project [codegen id : 49] Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22] Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_customer_sk#18, c_birth_year#22] -(128) ReusedExchange [Reuses operator id: 17] +(131) ReusedExchange [Reuses operator id: 17] Output [2]: [i_item_sk#16, i_item_id#17] -(129) BroadcastHashJoin [codegen id : 49] +(132) BroadcastHashJoin [codegen id : 49] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(130) Project [codegen id : 49] +(133) Project [codegen id : 49] Output [8]: [i_item_id#17, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22, i_item_sk#16, i_item_id#17] -(131) HashAggregate [codegen id : 49] +(134) HashAggregate [codegen id : 49] Input [8]: [i_item_id#17, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys [1]: [i_item_id#17] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#164, count#165, sum#166, count#167, sum#168, count#169, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177] Results [15]: [i_item_id#17, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191] -(132) Exchange +(135) Exchange Input [15]: [i_item_id#17, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=20] -(133) HashAggregate [codegen id : 50] +(136) HashAggregate [codegen id : 50] Input [15]: [i_item_id#17, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191] Keys [1]: [i_item_id#17] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] Aggregate Attributes [7]: [avg(agg1#28)#192, avg(agg2#29)#193, avg(agg3#30)#194, avg(agg4#31)#195, avg(agg5#32)#196, avg(agg6#33)#197, avg(agg7#34)#198] Results [11]: [i_item_id#17, null AS ca_country#199, null AS ca_state#200, null AS county#201, avg(agg1#28)#192 AS agg1#202, avg(agg2#29)#193 AS agg2#203, avg(agg3#30)#194 AS agg3#204, avg(agg4#31)#195 AS agg4#205, avg(agg5#32)#196 AS agg5#206, avg(agg6#33)#197 AS agg6#207, avg(agg7#34)#198 AS agg7#208] -(134) Scan parquet spark_catalog.default.catalog_sales +(137) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -779,138 +795,142 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(135) ColumnarToRow [codegen id : 57] +(138) ColumnarToRow [codegen id : 57] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(136) Filter [codegen id : 57] +(139) Filter [codegen id : 57] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(137) ReusedExchange [Reuses operator id: 8] +(140) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(138) BroadcastHashJoin [codegen id : 57] +(141) BroadcastHashJoin [codegen id : 57] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(139) Project [codegen id : 57] +(142) Project [codegen id : 57] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -(140) ReusedExchange [Reuses operator id: 161] +(143) ReusedExchange [Reuses operator id: 165] Output [1]: [d_date_sk#15] -(141) BroadcastHashJoin [codegen id : 57] +(144) BroadcastHashJoin [codegen id : 57] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(142) Project [codegen id : 57] +(145) Project [codegen id : 57] Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#15] -(143) Scan parquet spark_catalog.default.item +(146) Scan parquet spark_catalog.default.item Output [1]: [i_item_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(144) ColumnarToRow [codegen id : 53] +(147) ColumnarToRow [codegen id : 53] Input [1]: [i_item_sk#16] -(145) Filter [codegen id : 53] +(148) Filter [codegen id : 53] Input [1]: [i_item_sk#16] Condition : isnotnull(i_item_sk#16) -(146) BroadcastExchange +(149) BroadcastExchange Input [1]: [i_item_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] -(147) BroadcastHashJoin [codegen id : 57] +(150) BroadcastHashJoin [codegen id : 57] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(148) Project [codegen id : 57] +(151) Project [codegen id : 57] Output [7]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] Input [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#16] -(149) ReusedExchange [Reuses operator id: 125] +(152) ReusedExchange [Reuses operator id: 128] Output [2]: [c_customer_sk#18, c_birth_year#22] -(150) BroadcastHashJoin [codegen id : 57] +(153) BroadcastHashJoin [codegen id : 57] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(151) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +(154) Project [codegen id : 57] +Output [7]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22] Input [9]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_customer_sk#18, c_birth_year#22] -(152) HashAggregate [codegen id : 57] +(155) Project [codegen id : 57] +Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#22 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +Input [7]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#22] + +(156) HashAggregate [codegen id : 57] Input [7]: [agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] Keys: [] Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] Aggregate Attributes [14]: [sum#209, count#210, sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222] Results [14]: [sum#223, count#224, sum#225, count#226, sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236] -(153) Exchange +(157) Exchange Input [14]: [sum#223, count#224, sum#225, count#226, sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=22] -(154) HashAggregate [codegen id : 58] +(158) HashAggregate [codegen id : 58] Input [14]: [sum#223, count#224, sum#225, count#226, sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236] Keys: [] Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] Aggregate Attributes [7]: [avg(agg1#28)#237, avg(agg2#29)#238, avg(agg3#30)#239, avg(agg4#31)#240, avg(agg5#32)#241, avg(agg6#33)#242, avg(agg7#34)#243] Results [11]: [null AS i_item_id#244, null AS ca_country#245, null AS ca_state#246, null AS county#247, avg(agg1#28)#237 AS agg1#248, avg(agg2#29)#238 AS agg2#249, avg(agg3#30)#239 AS agg3#250, avg(agg4#31)#240 AS agg4#251, avg(agg5#32)#241 AS agg5#252, avg(agg6#33)#242 AS agg6#253, avg(agg7#34)#243 AS agg7#254] -(155) Union +(159) Union -(156) TakeOrderedAndProject +(160) TakeOrderedAndProject Input [11]: [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] Arguments: 100, [ca_country#26 ASC NULLS FIRST, ca_state#25 ASC NULLS FIRST, ca_county#24 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [i_item_id#17, ca_country#26, ca_state#25, ca_county#24, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (161) -+- * Project (160) - +- * Filter (159) - +- * ColumnarToRow (158) - +- Scan parquet spark_catalog.default.date_dim (157) +BroadcastExchange (165) ++- * Project (164) + +- * Filter (163) + +- * ColumnarToRow (162) + +- Scan parquet spark_catalog.default.date_dim (161) -(157) Scan parquet spark_catalog.default.date_dim +(161) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#15, d_year#255] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(158) ColumnarToRow [codegen id : 1] +(162) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_year#255] -(159) Filter [codegen id : 1] +(163) Filter [codegen id : 1] Input [2]: [d_date_sk#15, d_year#255] Condition : ((isnotnull(d_year#255) AND (d_year#255 = 2001)) AND isnotnull(d_date_sk#15)) -(160) Project [codegen id : 1] +(164) Project [codegen id : 1] Output [1]: [d_date_sk#15] Input [2]: [d_date_sk#15, d_year#255] -(161) BroadcastExchange +(165) BroadcastExchange Input [1]: [d_date_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] -Subquery:2 Hosting operator id = 99 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 102 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 134 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt index df98d8cfc002c..5ec9b61c5a40b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt @@ -7,86 +7,87 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (13) HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (4) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_id] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #2 + WholeStageCodegen (4) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + 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 #4 + WholeStageCodegen (1) + Project [cd_demo_sk,cd_dep_count] + Filter [cd_gender,cd_education_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (11) + Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (8) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (7) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (10) + Sort [cd_demo_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [cd_demo_sk,cd_dep_count] - Filter [cd_gender,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (11) - Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (7) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + Exchange [cd_demo_sk] #9 + WholeStageCodegen (9) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk] WholeStageCodegen (28) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -94,45 +95,46 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (27) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (19) - Sort [cs_bill_customer_sk] - InputAdapter - ReusedExchange [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] #2 - InputAdapter - WholeStageCodegen (26) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (25) - Project [c_customer_sk,c_birth_year,ca_state,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (22) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #12 - WholeStageCodegen (21) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (20) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - WholeStageCodegen (24) - Sort [cd_demo_sk] - InputAdapter - ReusedExchange [cd_demo_sk] #9 + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_id] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (19) + Sort [cs_bill_customer_sk] + InputAdapter + ReusedExchange [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] #2 + InputAdapter + WholeStageCodegen (26) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #11 + WholeStageCodegen (25) + Project [c_customer_sk,c_birth_year,ca_state,ca_country] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (22) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #12 + WholeStageCodegen (21) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (20) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + WholeStageCodegen (24) + Sort [cd_demo_sk] + InputAdapter + ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (42) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -140,46 +142,47 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (41) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (33) - Sort [cs_bill_customer_sk] - InputAdapter - ReusedExchange [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] #2 - InputAdapter - WholeStageCodegen (40) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #15 - WholeStageCodegen (39) - Project [c_customer_sk,c_birth_year,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (36) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #16 - WholeStageCodegen (35) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (34) - Project [ca_address_sk,ca_country] - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - WholeStageCodegen (38) - Sort [cd_demo_sk] - InputAdapter - ReusedExchange [cd_demo_sk] #9 + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_id] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (33) + Sort [cs_bill_customer_sk] + InputAdapter + ReusedExchange [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] #2 + InputAdapter + WholeStageCodegen (40) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #15 + WholeStageCodegen (39) + Project [c_customer_sk,c_birth_year,ca_country] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (36) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #16 + WholeStageCodegen (35) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (34) + Project [ca_address_sk,ca_country] + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + WholeStageCodegen (38) + Sort [cd_demo_sk] + InputAdapter + ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (50) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -239,28 +242,29 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (57) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (53) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (53) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_birth_year] #19 + Scan parquet spark_catalog.default.item [i_item_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_birth_year] #19 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 160a74a3b4816..e52f9d51204ba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -1,65 +1,67 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (34) - : : +- Exchange (33) - : : +- * Project (32) - : : +- * Filter (31) - : : +- * SortMergeJoin ExistenceJoin(exists#1) (30) - : : :- * SortMergeJoin ExistenceJoin(exists#2) (22) - : : : :- * SortMergeJoin LeftSemi (14) - : : : : :- * Sort (5) - : : : : : +- Exchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- * Sort (13) - : : : : +- Exchange (12) - : : : : +- * Project (11) - : : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet spark_catalog.default.store_sales (6) - : : : : +- ReusedExchange (9) - : : : +- * Sort (21) - : : : +- Exchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * ColumnarToRow (16) - : : : : +- Scan parquet spark_catalog.default.web_sales (15) - : : : +- ReusedExchange (17) - : : +- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- * Sort (39) - : +- Exchange (38) - : +- * Filter (37) - : +- * ColumnarToRow (36) - : +- Scan parquet spark_catalog.default.customer_address (35) - +- * Sort (48) - +- Exchange (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet spark_catalog.default.customer_demographics (44) +TakeOrderedAndProject (56) ++- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * Project (52) + +- * Filter (51) + +- * SortMergeJoin ExistenceJoin(exists#1) (50) + :- * SortMergeJoin ExistenceJoin(exists#2) (42) + : :- * Sort (34) + : : +- Exchange (33) + : : +- * Project (32) + : : +- * SortMergeJoin Inner (31) + : : :- * Sort (25) + : : : +- Exchange (24) + : : : +- * Project (23) + : : : +- * SortMergeJoin Inner (22) + : : : :- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * SortMergeJoin LeftSemi (14) + : : : : :- * Sort (5) + : : : : : +- Exchange (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.customer (1) + : : : : +- * Sort (13) + : : : : +- Exchange (12) + : : : : +- * Project (11) + : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet spark_catalog.default.store_sales (6) + : : : : +- ReusedExchange (9) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet spark_catalog.default.customer_address (17) + : : +- * Sort (30) + : : +- Exchange (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet spark_catalog.default.customer_demographics (26) + : +- * Sort (41) + : +- Exchange (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * ColumnarToRow (36) + : : +- Scan parquet spark_catalog.default.web_sales (35) + : +- ReusedExchange (37) + +- * Sort (49) + +- Exchange (48) + +- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * ColumnarToRow (44) + : +- Scan parquet spark_catalog.default.catalog_sales (43) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -67,7 +69,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : ((isnotnull(c_customer_sk#3) AND isnotnull(c_current_addr_sk#5)) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -92,7 +94,7 @@ Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Condition : isnotnull(ss_customer_sk#6) -(9) ReusedExchange [Reuses operator id: 59] +(9) ReusedExchange [Reuses operator id: 61] Output [1]: [d_date_sk#9] (10) BroadcastHashJoin [codegen id : 4] @@ -119,224 +121,232 @@ Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(15) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct +(15) Exchange +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(16) ColumnarToRow [codegen id : 8] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(16) Sort [codegen id : 7] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(17) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#12] +(17) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#10, ca_state#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(18) ColumnarToRow [codegen id : 8] +Input [2]: [ca_address_sk#10, ca_state#11] -(19) Project [codegen id : 8] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] +(19) Filter [codegen id : 8] +Input [2]: [ca_address_sk#10, ca_state#11] +Condition : isnotnull(ca_address_sk#10) (20) Exchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: hashpartitioning(ws_bill_customer_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] (21) Sort [codegen id : 9] -Input [1]: [ws_bill_customer_sk#10] -Arguments: [ws_bill_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 (22) SortMergeJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#10] +Join type: Inner Join condition: None -(23) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct +(23) Project [codegen id : 10] +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#10, ca_state#11] -(24) ColumnarToRow [codegen id : 12] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(24) Exchange +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(25) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#15] +(25) Sort [codegen id : 11] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(26) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(26) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(27) Project [codegen id : 12] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +(27) ColumnarToRow [codegen id : 12] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] -(28) Exchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: hashpartitioning(cs_ship_customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(28) Filter [codegen id : 12] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Condition : isnotnull(cd_demo_sk#12) -(29) Sort [codegen id : 13] -Input [1]: [cs_ship_customer_sk#13] -Arguments: [cs_ship_customer_sk#13 ASC NULLS FIRST], false, 0 +(29) Exchange +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: hashpartitioning(cd_demo_sk#12, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(30) SortMergeJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None +(30) Sort [codegen id : 13] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: [cd_demo_sk#12 ASC NULLS FIRST], false, 0 -(31) Filter [codegen id : 14] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) +(31) SortMergeJoin [codegen id : 14] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#12] +Join type: Inner +Join condition: None (32) Project [codegen id : 14] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] +Output [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11, cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] (33) Exchange -Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: hashpartitioning(c_customer_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=7] (34) Sort [codegen id : 15] -Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: [c_customer_sk#3 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +(35) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 16] -Input [2]: [ca_address_sk#16, ca_state#17] - -(37) Filter [codegen id : 16] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#8)] +ReadSchema: struct -(38) Exchange -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: hashpartitioning(ca_address_sk#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) ColumnarToRow [codegen id : 17] +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(39) Sort [codegen id : 17] -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16 ASC NULLS FIRST], false, 0 +(37) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#20] -(40) SortMergeJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] +(38) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 18] -Output [2]: [c_current_cdemo_sk#4, ca_state#17] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] +(39) Project [codegen id : 17] +Output [1]: [ws_bill_customer_sk#18] +Input [3]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20] -(42) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#17] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(40) Exchange +Input [1]: [ws_bill_customer_sk#18] +Arguments: hashpartitioning(ws_bill_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(43) Sort [codegen id : 19] -Input [2]: [c_current_cdemo_sk#4, ca_state#17] -Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 18] +Input [1]: [ws_bill_customer_sk#18] +Arguments: [ws_bill_customer_sk#18 ASC NULLS FIRST], false, 0 + +(42) SortMergeJoin [codegen id : 19] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ws_bill_customer_sk#18] +Join type: ExistenceJoin(exists#2) +Join condition: None -(44) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(43) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#8)] +ReadSchema: struct -(45) ColumnarToRow [codegen id : 20] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(44) ColumnarToRow [codegen id : 21] +Input [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] -(46) Filter [codegen id : 20] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Condition : isnotnull(cd_demo_sk#18) +(45) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#23] -(47) Exchange -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: hashpartitioning(cd_demo_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(46) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [cs_sold_date_sk#22] +Right keys [1]: [d_date_sk#23] +Join type: Inner +Join condition: None -(48) Sort [codegen id : 21] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: [cd_demo_sk#18 ASC NULLS FIRST], false, 0 +(47) Project [codegen id : 21] +Output [1]: [cs_ship_customer_sk#21] +Input [3]: [cs_ship_customer_sk#21, cs_sold_date_sk#22, d_date_sk#23] -(49) SortMergeJoin [codegen id : 22] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner +(48) Exchange +Input [1]: [cs_ship_customer_sk#21] +Arguments: hashpartitioning(cs_ship_customer_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(49) Sort [codegen id : 22] +Input [1]: [cs_ship_customer_sk#21] +Arguments: [cs_ship_customer_sk#21 ASC NULLS FIRST], false, 0 + +(50) SortMergeJoin [codegen id : 23] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#21] +Join type: ExistenceJoin(exists#1) Join condition: None -(50) Project [codegen id : 22] -Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(51) Filter [codegen id : 23] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] +Condition : (exists#2 OR exists#1) + +(52) Project [codegen id : 23] +Output [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] -(51) HashAggregate [codegen id : 22] -Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] +(53) HashAggregate [codegen id : 23] +Input [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#15), partial_max(cd_dep_count#15), partial_sum(cd_dep_count#15), partial_avg(cd_dep_employed_count#16), partial_max(cd_dep_employed_count#16), partial_sum(cd_dep_employed_count#16), partial_avg(cd_dep_college_count#17), partial_max(cd_dep_college_count#17), partial_sum(cd_dep_college_count#17)] Aggregate Attributes [13]: [count#24, sum#25, count#26, max#27, sum#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36] -Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] +Results [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -(52) Exchange -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(54) Exchange +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] +Arguments: hashpartitioning(ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(53) HashAggregate [codegen id : 23] -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] -Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#21)#51, max(cd_dep_count#21)#52, sum(cd_dep_count#21)#53, avg(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, sum(cd_dep_employed_count#22)#56, avg(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, sum(cd_dep_college_count#23)#59] -Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, count(1)#50 AS cnt1#60, avg(cd_dep_count#21)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#21)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#22)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#22)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#23)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#23)#59 AS sum(cd_dep_college_count)#71] +(55) HashAggregate [codegen id : 24] +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [count(1), avg(cd_dep_count#15), max(cd_dep_count#15), sum(cd_dep_count#15), avg(cd_dep_employed_count#16), max(cd_dep_employed_count#16), sum(cd_dep_employed_count#16), avg(cd_dep_college_count#17), max(cd_dep_college_count#17), sum(cd_dep_college_count#17)] +Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#15)#51, max(cd_dep_count#15)#52, sum(cd_dep_count#15)#53, avg(cd_dep_employed_count#16)#54, max(cd_dep_employed_count#16)#55, sum(cd_dep_employed_count#16)#56, avg(cd_dep_college_count#17)#57, max(cd_dep_college_count#17)#58, sum(cd_dep_college_count#17)#59] +Results [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, count(1)#50 AS cnt1#60, avg(cd_dep_count#15)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#15)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#15)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#16, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#16)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#16)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#16)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#17, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#17)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#17)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#17)#59 AS sum(cd_dep_college_count)#71] -(54) TakeOrderedAndProject -Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] -Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] +(56) TakeOrderedAndProject +Input [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] +Arguments: 100, [ca_state#11 ASC NULLS FIRST, cd_gender#13 ASC NULLS FIRST, cd_marital_status#14 ASC NULLS FIRST, cd_dep_count#15 ASC NULLS FIRST, cd_dep_employed_count#16 ASC NULLS FIRST, cd_dep_college_count#17 ASC NULLS FIRST], [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet spark_catalog.default.date_dim (55) +BroadcastExchange (61) ++- * Project (60) + +- * Filter (59) + +- * ColumnarToRow (58) + +- Scan parquet spark_catalog.default.date_dim (57) -(55) Scan parquet spark_catalog.default.date_dim +(57) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#72, d_qoy#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] +(58) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(57) Filter [codegen id : 1] +(59) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Condition : ((((isnotnull(d_year#72) AND isnotnull(d_qoy#73)) AND (d_year#72 = 2002)) AND (d_qoy#73 < 4)) AND isnotnull(d_date_sk#9)) -(58) Project [codegen id : 1] +(60) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] -(59) BroadcastExchange +(61) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 35 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index e0538113b2fdf..68a937b956dd2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -1,112 +1,116 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (23) + WholeStageCodegen (24) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (22) + WholeStageCodegen (23) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (19) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #2 - WholeStageCodegen (18) - Project [c_current_cdemo_sk,ca_state] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (15) - Sort [c_current_addr_sk] - InputAdapter - Exchange [c_current_addr_sk] #3 - WholeStageCodegen (14) - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - SortMergeJoin [c_customer_sk,cs_ship_customer_sk] - InputAdapter - WholeStageCodegen (10) - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Filter [exists,exists] + SortMergeJoin [c_customer_sk,cs_ship_customer_sk] + InputAdapter + WholeStageCodegen (19) + SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + InputAdapter + WholeStageCodegen (15) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #2 + WholeStageCodegen (14) + Project [c_customer_sk,ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #3 + WholeStageCodegen (10) + Project [c_customer_sk,c_current_cdemo_sk,ca_state] + SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (6) - SortMergeJoin [c_customer_sk,ss_customer_sk] - InputAdapter - WholeStageCodegen (2) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #4 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + WholeStageCodegen (7) + Sort [c_current_addr_sk] InputAdapter - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #5 - WholeStageCodegen (4) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_qoy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #6 + Exchange [c_current_addr_sk] #4 + WholeStageCodegen (6) + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (2) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #5 + WholeStageCodegen (1) + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #6 + WholeStageCodegen (4) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_qoy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter WholeStageCodegen (9) - Sort [ws_bill_customer_sk] + Sort [ca_address_sk] InputAdapter - Exchange [ws_bill_customer_sk] #7 + Exchange [ca_address_sk] #8 WholeStageCodegen (8) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Filter [ca_address_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (13) - Sort [cs_ship_customer_sk] + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + WholeStageCodegen (13) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #9 + WholeStageCodegen (12) + Filter [cd_demo_sk] + ColumnarToRow InputAdapter - Exchange [cs_ship_customer_sk] #8 - WholeStageCodegen (12) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + InputAdapter + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #10 WholeStageCodegen (17) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #9 - WholeStageCodegen (16) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - WholeStageCodegen (21) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #10 - WholeStageCodegen (20) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + WholeStageCodegen (22) + Sort [cs_ship_customer_sk] + InputAdapter + Exchange [cs_ship_customer_sk] #11 + WholeStageCodegen (21) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index af84a241d9c92..aa1d34a1ad151 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -4,51 +4,51 @@ TakeOrderedAndProject (43) +- Exchange (41) +- * HashAggregate (40) +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * Filter (26) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (25) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (18) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (10) - : : : : +- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (7) - : : : +- BroadcastExchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * ColumnarToRow (13) - : : : : +- Scan parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet spark_catalog.default.customer_demographics (34) + +- * Filter (38) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (37) + :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (30) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (11) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (7) + : : : +- BroadcastExchange (15) + : : : +- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet spark_catalog.default.customer_address (12) + : : +- BroadcastExchange (21) + : : +- * Filter (20) + : : +- * ColumnarToRow (19) + : : +- Scan parquet spark_catalog.default.customer_demographics (18) + : +- BroadcastExchange (29) + : +- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * ColumnarToRow (25) + : : +- Scan parquet spark_catalog.default.web_sales (24) + : +- ReusedExchange (26) + +- BroadcastExchange (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (32) + : +- Scan parquet spark_catalog.default.catalog_sales (31) + +- ReusedExchange (33) (1) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -56,7 +56,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : ((isnotnull(c_customer_sk#3) AND isnotnull(c_current_addr_sk#5)) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] @@ -96,157 +96,157 @@ Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi Join condition: None -(12) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(12) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#10, ca_state#11] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct + +(13) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#10, ca_state#11] -(13) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) Filter [codegen id : 3] +Input [2]: [ca_address_sk#10, ca_state#11] +Condition : isnotnull(ca_address_sk#10) -(14) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#12] +(15) BroadcastExchange +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] +(16) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] +(17) Project [codegen id : 9] +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#10, ca_state#11] -(17) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) +(19) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] + +(20) Filter [codegen id : 4] +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Condition : isnotnull(cd_demo_sk#12) + +(21) BroadcastExchange +Input [6]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#12] +Join type: Inner Join condition: None -(19) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(23) Project [codegen id : 9] +Output [7]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, c_current_cdemo_sk#4, ca_state#11, cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] + +(24) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#8)] +ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(25) ColumnarToRow [codegen id : 6] +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(21) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#15] +(26) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#20] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +(28) Project [codegen id : 6] +Output [1]: [ws_bill_customer_sk#18] +Input [3]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20] -(24) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(29) BroadcastExchange +Input [1]: [ws_bill_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) +Right keys [1]: [ws_bill_customer_sk#18] +Join type: ExistenceJoin(exists#2) Join condition: None -(26) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(27) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(28) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +(31) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#17] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#8)] +ReadSchema: struct -(30) Filter [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) +(32) ColumnarToRow [codegen id : 8] +Input [2]: [cs_ship_customer_sk#21, cs_sold_date_sk#22] -(31) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(33) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#23] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#22] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#17] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#17] +(35) Project [codegen id : 8] +Output [1]: [cs_ship_customer_sk#21] +Input [3]: [cs_ship_customer_sk#21, cs_sold_date_sk#22, d_date_sk#23] -(34) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +(36) BroadcastExchange +Input [1]: [cs_ship_customer_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Condition : isnotnull(cd_demo_sk#18) - -(37) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#21] +Join type: ExistenceJoin(exists#1) Join condition: None +(38) Filter [codegen id : 9] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] +Condition : (exists#2 OR exists#1) + (39) Project [codegen id : 9] -Output [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Input [8]: [c_current_cdemo_sk#4, ca_state#17, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] +Output [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Input [9]: [c_customer_sk#3, ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, exists#2, exists#1] (40) HashAggregate [codegen id : 9] -Input [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] +Input [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#15), partial_max(cd_dep_count#15), partial_sum(cd_dep_count#15), partial_avg(cd_dep_employed_count#16), partial_max(cd_dep_employed_count#16), partial_sum(cd_dep_employed_count#16), partial_avg(cd_dep_college_count#17), partial_max(cd_dep_college_count#17), partial_sum(cd_dep_college_count#17)] Aggregate Attributes [13]: [count#24, sum#25, count#26, max#27, sum#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36] -Results [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] +Results [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] (41) Exchange -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -Arguments: hashpartitioning(ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] +Arguments: hashpartitioning(ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, 5), ENSURE_REQUIREMENTS, [plan_id=6] (42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] -Keys [6]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] -Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#21)#51, max(cd_dep_count#21)#52, sum(cd_dep_count#21)#53, avg(cd_dep_employed_count#22)#54, max(cd_dep_employed_count#22)#55, sum(cd_dep_employed_count#22)#56, avg(cd_dep_college_count#23)#57, max(cd_dep_college_count#23)#58, sum(cd_dep_college_count#23)#59] -Results [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, count(1)#50 AS cnt1#60, avg(cd_dep_count#21)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#21)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#21)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#22, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#22)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#22)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#22)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#23, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#23)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#23)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#23)#59 AS sum(cd_dep_college_count)#71] +Input [19]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17, count#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49] +Keys [6]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cd_dep_employed_count#16, cd_dep_college_count#17] +Functions [10]: [count(1), avg(cd_dep_count#15), max(cd_dep_count#15), sum(cd_dep_count#15), avg(cd_dep_employed_count#16), max(cd_dep_employed_count#16), sum(cd_dep_employed_count#16), avg(cd_dep_college_count#17), max(cd_dep_college_count#17), sum(cd_dep_college_count#17)] +Aggregate Attributes [10]: [count(1)#50, avg(cd_dep_count#15)#51, max(cd_dep_count#15)#52, sum(cd_dep_count#15)#53, avg(cd_dep_employed_count#16)#54, max(cd_dep_employed_count#16)#55, sum(cd_dep_employed_count#16)#56, avg(cd_dep_college_count#17)#57, max(cd_dep_college_count#17)#58, sum(cd_dep_college_count#17)#59] +Results [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, count(1)#50 AS cnt1#60, avg(cd_dep_count#15)#51 AS avg(cd_dep_count)#61, max(cd_dep_count#15)#52 AS max(cd_dep_count)#62, sum(cd_dep_count#15)#53 AS sum(cd_dep_count)#63, cd_dep_employed_count#16, count(1)#50 AS cnt2#64, avg(cd_dep_employed_count#16)#54 AS avg(cd_dep_employed_count)#65, max(cd_dep_employed_count#16)#55 AS max(cd_dep_employed_count)#66, sum(cd_dep_employed_count#16)#56 AS sum(cd_dep_employed_count)#67, cd_dep_college_count#17, count(1)#50 AS cnt3#68, avg(cd_dep_college_count#17)#57 AS avg(cd_dep_college_count)#69, max(cd_dep_college_count#17)#58 AS max(cd_dep_college_count)#70, sum(cd_dep_college_count#17)#59 AS sum(cd_dep_college_count)#71] (43) TakeOrderedAndProject -Input [18]: [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] -Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#22, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#23, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] +Input [18]: [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] +Arguments: 100, [ca_state#11 ASC NULLS FIRST, cd_gender#13 ASC NULLS FIRST, cd_marital_status#14 ASC NULLS FIRST, cd_dep_count#15 ASC NULLS FIRST, cd_dep_employed_count#16 ASC NULLS FIRST, cd_dep_college_count#17 ASC NULLS FIRST], [ca_state#11, cd_gender#13, cd_marital_status#14, cd_dep_count#15, cnt1#60, avg(cd_dep_count)#61, max(cd_dep_count)#62, sum(cd_dep_count)#63, cd_dep_employed_count#16, cnt2#64, avg(cd_dep_employed_count)#65, max(cd_dep_employed_count)#66, sum(cd_dep_employed_count)#67, cd_dep_college_count#17, cnt3#68, avg(cd_dep_college_count)#69, max(cd_dep_college_count)#70, sum(cd_dep_college_count)#71] ===== Subqueries ===== @@ -280,8 +280,8 @@ Input [3]: [d_date_sk#9, d_year#72, d_qoy#73] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 24 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 35629eada6ab1..2e05422cfe28b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (9) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_sk,ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_customer_sk,c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -39,37 +39,37 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + WholeStageCodegen (3) + Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - Filter [ca_address_sk] + WholeStageCodegen (6) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 15e58db64a798..caaacfeb8586a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -58,7 +58,7 @@ TakeOrderedAndProject (53) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -66,7 +66,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : ((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 2751f923bd2d3..d5bee1986251d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 362cc13721c83..90d7897574ab3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject (43) Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -56,7 +56,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : ((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet spark_catalog.default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index 18d39e8d1cc76..56e66713c1390 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] 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 8ecffb517bc14..497fe172c9bf6 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 (91) -+- * HashAggregate (90) - +- Exchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- Union (73) +TakeOrderedAndProject (90) ++- * HashAggregate (89) + +- Exchange (88) + +- * HashAggregate (87) + +- Union (86) + :- * HashAggregate (75) + : +- Exchange (74) + : +- * HashAggregate (73) + : +- Union (72) : :- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) @@ -50,46 +50,45 @@ TakeOrderedAndProject (91) : : : +- * ColumnarToRow (32) : : : +- Scan parquet spark_catalog.default.catalog_page (31) : : +- ReusedExchange (37) - : +- * HashAggregate (72) - : +- Exchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- Union (60) + : +- * HashAggregate (71) + : +- Exchange (70) + : +- * HashAggregate (69) + : +- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- Union (59) : : : :- * Project (46) : : : : +- * Filter (45) : : : : +- * ColumnarToRow (44) : : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : : +- * 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) + : : : +- * 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) (1) Scan parquet spark_catalog.default.store_sales @@ -160,7 +159,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: 96] +(16) ReusedExchange [Reuses operator id: 95] Output [1]: [d_date_sk#24] (17) BroadcastHashJoin [codegen id : 5] @@ -259,7 +258,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: 96] +(37) ReusedExchange [Reuses operator id: 95] Output [1]: [d_date_sk#64] (38) BroadcastHashJoin [codegen id : 11] @@ -314,250 +313,245 @@ 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) 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 +(49) 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] -(51) Sort [codegen id : 15] +(50) 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 -(52) Scan parquet spark_catalog.default.web_sales +(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 -(53) ColumnarToRow [codegen id : 16] +(52) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(54) Filter [codegen id : 16] +(53) 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)) -(55) Project [codegen id : 16] +(54) 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] -(56) Exchange +(55) 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] -(57) Sort [codegen id : 17] +(56) 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 -(58) SortMergeJoin [codegen id : 18] +(57) 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 -(59) Project [codegen id : 18] +(58) 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] -(60) Union +(59) Union -(61) Scan parquet spark_catalog.default.web_site +(60) 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 -(62) ColumnarToRow [codegen id : 19] +(61) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] -(63) Filter [codegen id : 19] +(62) Filter [codegen id : 19] Input [2]: [web_site_sk#107, web_site_id#108] Condition : isnotnull(web_site_sk#107) -(64) BroadcastExchange +(63) 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] -(65) BroadcastHashJoin [codegen id : 21] +(64) 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 -(66) Project [codegen id : 21] +(65) 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] -(67) ReusedExchange [Reuses operator id: 96] +(66) ReusedExchange [Reuses operator id: 95] Output [1]: [d_date_sk#109] -(68) BroadcastHashJoin [codegen id : 21] +(67) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(69) Project [codegen id : 21] +(68) 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] -(70) HashAggregate [codegen id : 21] +(69) 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] -(71) Exchange +(70) 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] -(72) HashAggregate [codegen id : 22] +(71) 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] -(73) Union +(72) Union -(74) HashAggregate [codegen id : 23] +(73) 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] -(75) Exchange +(74) 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] -(76) HashAggregate [codegen id : 24] +(75) 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] -(77) ReusedExchange [Reuses operator id: 75] +(76) ReusedExchange [Reuses operator id: 74] Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(78) HashAggregate [codegen id : 48] +(77) HashAggregate [codegen id : 48] 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 [4]: [channel#37, sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(79) HashAggregate [codegen id : 48] +(78) HashAggregate [codegen id : 48] Input [4]: [channel#37, sales#145, returns#146, profit#147] Keys [1]: [channel#37] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] Results [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -(80) Exchange +(79) Exchange Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(81) HashAggregate [codegen id : 49] +(80) HashAggregate [codegen id : 49] Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Keys [1]: [channel#37] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#160, sum(returns#146)#161, sum(profit#147)#162] Results [5]: [channel#37, null AS id#163, sum(sales#145)#160 AS sum(sales)#164, sum(returns#146)#161 AS sum(returns)#165, sum(profit#147)#162 AS sum(profit)#166] -(82) ReusedExchange [Reuses operator id: 75] +(81) ReusedExchange [Reuses operator id: 74] Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(83) HashAggregate [codegen id : 73] +(82) HashAggregate [codegen id : 73] 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 [3]: [sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(84) HashAggregate [codegen id : 73] +(83) HashAggregate [codegen id : 73] Input [3]: [sales#145, returns#146, profit#147] Keys: [] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] Results [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(85) Exchange +(84) Exchange Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(86) HashAggregate [codegen id : 74] +(85) HashAggregate [codegen id : 74] Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys: [] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#179, sum(returns#146)#180, sum(profit#147)#181] Results [5]: [null AS channel#182, null AS id#183, sum(sales#145)#179 AS sum(sales)#184, sum(returns#146)#180 AS sum(returns)#185, sum(profit#147)#181 AS sum(profit)#186] -(87) Union +(86) Union -(88) HashAggregate [codegen id : 75] +(87) 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] -(89) Exchange +(88) 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] -(90) HashAggregate [codegen id : 76] +(89) 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] -(91) TakeOrderedAndProject +(90) 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 (96) -+- * Project (95) - +- * Filter (94) - +- * ColumnarToRow (93) - +- Scan parquet spark_catalog.default.date_dim (92) +BroadcastExchange (95) ++- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet spark_catalog.default.date_dim (91) -(92) Scan parquet spark_catalog.default.date_dim +(91) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_date#187] 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 -(93) ColumnarToRow [codegen id : 1] +(92) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_date#187] -(94) Filter [codegen id : 1] +(93) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_date#187] Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 1998-08-18)) AND isnotnull(d_date_sk#24)) -(95) Project [codegen id : 1] +(94) Project [codegen id : 1] Output [1]: [d_date_sk#24] Input [2]: [d_date_sk#24, d_date#187] -(96) BroadcastExchange +(95) 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 33a7452ff905d..f8cc4833beed6 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,11 +120,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #9 WholeStageCodegen (14) - 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 + 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 2408f9eb3c714..04d9e50272b3c 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 (88) -+- * HashAggregate (87) - +- Exchange (86) - +- * HashAggregate (85) - +- Union (84) - :- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- Union (70) +TakeOrderedAndProject (87) ++- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- Union (83) + :- * HashAggregate (72) + : +- Exchange (71) + : +- * HashAggregate (70) + : +- Union (69) : :- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) @@ -50,43 +50,42 @@ TakeOrderedAndProject (88) : : +- * Filter (36) : : +- * ColumnarToRow (35) : : +- Scan parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (69) - : +- Exchange (68) - : +- * HashAggregate (67) - : +- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- Union (57) + : +- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- Union (56) : : : :- * Project (46) : : : : +- * Filter (45) : : : : +- * ColumnarToRow (44) : : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : : +- * 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) + : : : +- * 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) (1) Scan parquet spark_catalog.default.store_sales @@ -129,7 +128,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: 93] +(10) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#22] (11) BroadcastHashJoin [codegen id : 5] @@ -228,7 +227,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: 93] +(31) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#62] (32) BroadcastHashJoin [codegen id : 11] @@ -311,238 +310,233 @@ 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) Filter [codegen id : 14] +(49) BroadcastExchange 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)) +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(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 +(50) 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 +(51) ColumnarToRow Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -(53) Filter +(52) 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)) -(54) Project +(53) 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] -(55) BroadcastHashJoin [codegen id : 15] +(54) 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 -(56) Project [codegen id : 15] +(55) 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] -(57) Union +(56) Union -(58) ReusedExchange [Reuses operator id: 93] +(57) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#107] -(59) BroadcastHashJoin [codegen id : 18] +(58) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#87] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(60) Project [codegen id : 18] +(59) 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] -(61) Scan parquet spark_catalog.default.web_site +(60) 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 -(62) ColumnarToRow [codegen id : 17] +(61) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] -(63) Filter [codegen id : 17] +(62) Filter [codegen id : 17] Input [2]: [web_site_sk#108, web_site_id#109] Condition : isnotnull(web_site_sk#108) -(64) BroadcastExchange +(63) 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] -(65) BroadcastHashJoin [codegen id : 18] +(64) 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 -(66) Project [codegen id : 18] +(65) 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] -(67) HashAggregate [codegen id : 18] +(66) 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] -(68) Exchange +(67) 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] -(69) HashAggregate [codegen id : 19] +(68) 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] -(70) Union +(69) Union -(71) HashAggregate [codegen id : 20] +(70) 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] -(72) Exchange +(71) 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] -(73) HashAggregate [codegen id : 21] +(72) 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] -(74) ReusedExchange [Reuses operator id: 72] +(73) ReusedExchange [Reuses operator id: 71] Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(75) HashAggregate [codegen id : 42] +(74) HashAggregate [codegen id : 42] 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 [4]: [channel#37, sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(76) HashAggregate [codegen id : 42] +(75) HashAggregate [codegen id : 42] Input [4]: [channel#37, sales#145, returns#146, profit#147] Keys [1]: [channel#37] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] Results [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -(77) Exchange +(76) Exchange Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(78) HashAggregate [codegen id : 43] +(77) HashAggregate [codegen id : 43] Input [7]: [channel#37, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] Keys [1]: [channel#37] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#160, sum(returns#146)#161, sum(profit#147)#162] Results [5]: [channel#37, null AS id#163, sum(sales#145)#160 AS sum(sales)#164, sum(returns#146)#161 AS sum(returns)#165, sum(profit#147)#162 AS sum(profit)#166] -(79) ReusedExchange [Reuses operator id: 72] +(78) ReusedExchange [Reuses operator id: 71] Output [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -(80) HashAggregate [codegen id : 64] +(79) HashAggregate [codegen id : 64] 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 [3]: [sum(sales#39)#139 AS sales#145, sum(returns#40)#140 AS returns#146, sum(profit#41)#141 AS profit#147] -(81) HashAggregate [codegen id : 64] +(80) HashAggregate [codegen id : 64] Input [3]: [sales#145, returns#146, profit#147] Keys: [] Functions [3]: [partial_sum(sales#145), partial_sum(returns#146), partial_sum(profit#147)] Aggregate Attributes [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] Results [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -(82) Exchange +(81) Exchange Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(83) HashAggregate [codegen id : 65] +(82) HashAggregate [codegen id : 65] Input [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] Keys: [] Functions [3]: [sum(sales#145), sum(returns#146), sum(profit#147)] Aggregate Attributes [3]: [sum(sales#145)#179, sum(returns#146)#180, sum(profit#147)#181] Results [5]: [null AS channel#182, null AS id#183, sum(sales#145)#179 AS sum(sales)#184, sum(returns#146)#180 AS sum(returns)#185, sum(profit#147)#181 AS sum(profit)#186] -(84) Union +(83) Union -(85) HashAggregate [codegen id : 66] +(84) 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] -(86) Exchange +(85) 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] -(87) HashAggregate [codegen id : 67] +(86) 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] -(88) TakeOrderedAndProject +(87) 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 (93) -+- * Project (92) - +- * Filter (91) - +- * ColumnarToRow (90) - +- Scan parquet spark_catalog.default.date_dim (89) +BroadcastExchange (92) ++- * Project (91) + +- * Filter (90) + +- * ColumnarToRow (89) + +- Scan parquet spark_catalog.default.date_dim (88) -(89) Scan parquet spark_catalog.default.date_dim +(88) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#187] 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 -(90) ColumnarToRow [codegen id : 1] +(89) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_date#187] -(91) Filter [codegen id : 1] +(90) Filter [codegen id : 1] Input [2]: [d_date_sk#22, d_date#187] Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(92) Project [codegen id : 1] +(91) Project [codegen id : 1] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#187] -(93) BroadcastExchange +(92) 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 7c8f9566db242..c7d72bb6c5bd7 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,11 +117,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (14) - 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 + 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-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 1546b2928c357..ea6ab7567d79f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -90,7 +90,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 8] @@ -98,7 +98,7 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] (9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) +Condition : (isnotnull(s_state#8) AND isnotnull(s_store_sk#6)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index f1d0b073c0210..7f964d2aa30e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 6d5143bc3c4ea..6e7344e1b1719 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -90,7 +90,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 8] @@ -98,7 +98,7 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] (9) Filter [codegen id : 8] Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_state#8)) +Condition : (isnotnull(s_state#8) AND isnotnull(s_store_sk#6)) (10) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 459b5ab9dd70d..f6af4ac4e97f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_state,s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index 2342bfdeef3bd..82f731e5988f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * SortMergeJoin Inner (77) - :- * Project (59) - : +- * SortMergeJoin Inner (58) +TakeOrderedAndProject (81) ++- * Project (80) + +- * SortMergeJoin Inner (79) + :- * Project (60) + : +- * SortMergeJoin Inner (59) : :- * SortMergeJoin Inner (39) : : :- * Sort (21) : : : +- Exchange (20) @@ -43,41 +43,43 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (25) : : +- * Sort (31) : : +- ReusedExchange (30) - : +- * Sort (57) - : +- Exchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- Exchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * SortMergeJoin Inner (50) - : :- * Sort (47) - : : +- Exchange (46) - : : +- * Project (45) - : : +- * BroadcastHashJoin Inner BuildRight (44) - : : :- * Filter (42) - : : : +- * ColumnarToRow (41) - : : : +- Scan parquet spark_catalog.default.web_sales (40) - : : +- ReusedExchange (43) - : +- * Sort (49) - : +- ReusedExchange (48) - +- * Sort (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * SortMergeJoin Inner (70) - :- * Sort (67) - : +- Exchange (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Filter (62) - : : +- * ColumnarToRow (61) - : : +- Scan parquet spark_catalog.default.web_sales (60) - : +- ReusedExchange (63) - +- * Sort (69) - +- ReusedExchange (68) + : +- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- Exchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * SortMergeJoin Inner (50) + : :- * Sort (47) + : : +- Exchange (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- * Filter (42) + : : : +- * ColumnarToRow (41) + : : : +- Scan parquet spark_catalog.default.web_sales (40) + : : +- ReusedExchange (43) + : +- * Sort (49) + : +- ReusedExchange (48) + +- * Sort (78) + +- Exchange (77) + +- * Project (76) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * SortMergeJoin Inner (71) + :- * Sort (68) + : +- Exchange (67) + : +- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Filter (63) + : : +- * ColumnarToRow (62) + : : +- Scan parquet spark_catalog.default.web_sales (61) + : +- ReusedExchange (64) + +- * Sort (70) + +- ReusedExchange (69) (1) Scan parquet spark_catalog.default.store_sales @@ -95,7 +97,7 @@ Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 83] +(4) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#5, d_year#6] (5) BroadcastHashJoin [codegen id : 2] @@ -193,7 +195,7 @@ Input [3]: [ss_customer_sk#16, ss_net_paid#17, ss_sold_date_sk#18] Input [3]: [ss_customer_sk#16, ss_net_paid#17, ss_sold_date_sk#18] Condition : isnotnull(ss_customer_sk#16) -(25) ReusedExchange [Reuses operator id: 87] +(25) ReusedExchange [Reuses operator id: 89] Output [2]: [d_date_sk#20, d_year#21] (26) BroadcastHashJoin [codegen id : 10] @@ -278,7 +280,7 @@ Input [3]: [ws_bill_customer_sk#32, ws_net_paid#33, ws_sold_date_sk#34] Input [3]: [ws_bill_customer_sk#32, ws_net_paid#33, ws_sold_date_sk#34] Condition : isnotnull(ws_bill_customer_sk#32) -(43) ReusedExchange [Reuses operator id: 83] +(43) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#35, d_year#36] (44) BroadcastHashJoin [codegen id : 19] @@ -338,171 +340,179 @@ Results [2]: [c_customer_id#38 AS customer_id#44, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#44, year_total#45] Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) -(56) Exchange +(56) Project [codegen id : 24] +Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] Input [2]: [customer_id#44, year_total#45] -Arguments: hashpartitioning(customer_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(57) Sort [codegen id : 25] -Input [2]: [customer_id#44, year_total#45] -Arguments: [customer_id#44 ASC NULLS FIRST], false, 0 +(57) Exchange +Input [2]: [customer_id#46, year_total#47] +Arguments: hashpartitioning(customer_id#46, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(58) Sort [codegen id : 25] +Input [2]: [customer_id#46, year_total#47] +Arguments: [customer_id#46 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 26] +(59) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#44] +Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(59) Project [codegen id : 26] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#44, year_total#45] +(60) Project [codegen id : 26] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -(60) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +(61) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] +(62) ColumnarToRow [codegen id : 28] +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -(62) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48] -Condition : isnotnull(ws_bill_customer_sk#46) +(63) Filter [codegen id : 28] +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Condition : isnotnull(ws_bill_customer_sk#48) -(63) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#49, d_year#50] +(64) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#51, d_year#52] -(64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +(65) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ws_sold_date_sk#50] +Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(65) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50] -Input [5]: [ws_bill_customer_sk#46, ws_net_paid#47, ws_sold_date_sk#48, d_date_sk#49, d_year#50] +(66) Project [codegen id : 28] +Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52] +Input [5]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50, d_date_sk#51, d_year#52] -(66) Exchange -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50] -Arguments: hashpartitioning(ws_bill_customer_sk#46, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(67) Exchange +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52] +Arguments: hashpartitioning(ws_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(67) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50] -Arguments: [ws_bill_customer_sk#46 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 29] +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52] +Arguments: [ws_bill_customer_sk#48 ASC NULLS FIRST], false, 0 -(68) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +(69) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] -(69) Sort [codegen id : 31] -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 31] +Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +Arguments: [c_customer_sk#53 ASC NULLS FIRST], false, 0 -(70) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#46] -Right keys [1]: [c_customer_sk#51] +(71) SortMergeJoin [codegen id : 32] +Left keys [1]: [ws_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#53] Join type: Inner Join condition: None -(71) Project [codegen id : 32] -Output [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, ws_net_paid#47, d_year#50] -Input [7]: [ws_bill_customer_sk#46, ws_net_paid#47, d_year#50, c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] - -(72) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, ws_net_paid#47, d_year#50] -Keys [4]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#47))] -Aggregate Attributes [1]: [sum#55] -Results [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, sum#56] - -(73) Exchange -Input [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, sum#56] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(74) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50, sum#56] -Keys [4]: [c_customer_id#52, c_first_name#53, c_last_name#54, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_net_paid#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#47))#43] -Results [2]: [c_customer_id#52 AS customer_id#57, MakeDecimal(sum(UnscaledValue(ws_net_paid#47))#43,17,2) AS year_total#58] - -(75) Exchange -Input [2]: [customer_id#57, year_total#58] -Arguments: hashpartitioning(customer_id#57, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(76) Sort [codegen id : 34] -Input [2]: [customer_id#57, year_total#58] -Arguments: [customer_id#57 ASC NULLS FIRST], false, 0 - -(77) SortMergeJoin [codegen id : 35] +(72) Project [codegen id : 32] +Output [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, ws_net_paid#49, d_year#52] +Input [7]: [ws_bill_customer_sk#48, ws_net_paid#49, d_year#52, c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] + +(73) HashAggregate [codegen id : 32] +Input [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, ws_net_paid#49, d_year#52] +Keys [4]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#49))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, sum#58] + +(74) Exchange +Input [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, sum#58] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(75) HashAggregate [codegen id : 33] +Input [5]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52, sum#58] +Keys [4]: [c_customer_id#54, c_first_name#55, c_last_name#56, d_year#52] +Functions [1]: [sum(UnscaledValue(ws_net_paid#49))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#49))#43] +Results [2]: [c_customer_id#54 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#49))#43,17,2) AS year_total#60] + +(76) Project [codegen id : 33] +Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] +Input [2]: [customer_id#59, year_total#60] + +(77) Exchange +Input [2]: [customer_id#61, year_total#62] +Arguments: hashpartitioning(customer_id#61, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(78) Sort [codegen id : 34] +Input [2]: [customer_id#61, year_total#62] +Arguments: [customer_id#61 ASC NULLS FIRST], false, 0 + +(79) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#57] +Right keys [1]: [customer_id#61] Join type: Inner -Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#47 > 0.00) THEN (year_total#62 / year_total#47) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(78) Project [codegen id : 35] +(80) Project [codegen id : 35] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45, customer_id#57, year_total#58] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#61, year_total#62] -(79) TakeOrderedAndProject +(81) TakeOrderedAndProject Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (85) ++- * Filter (84) + +- * ColumnarToRow (83) + +- Scan parquet spark_catalog.default.date_dim (82) -(80) Scan parquet spark_catalog.default.date_dim +(82) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_year#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] +(83) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_year#6] -(82) Filter [codegen id : 1] +(84) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_year#6] Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) -(83) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (87) -+- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (89) ++- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet spark_catalog.default.date_dim (86) -(84) Scan parquet spark_catalog.default.date_dim +(86) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(86) Filter [codegen id : 1] +(88) Filter [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] Condition : (((isnotnull(d_year#21) AND (d_year#21 = 2002)) AND d_year#21 IN (2001,2002)) AND isnotnull(d_date_sk#20)) -(87) BroadcastExchange +(89) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#34 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#19 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt index 5546844311c36..4706eca24a0e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt @@ -99,64 +99,66 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter Exchange [customer_id] #10 WholeStageCodegen (24) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (20) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (19) - Project [ws_bill_customer_sk,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #4 - InputAdapter - WholeStageCodegen (22) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + WholeStageCodegen (23) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - WholeStageCodegen (32) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (28) - Project [ws_bill_customer_sk,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #9 - InputAdapter - WholeStageCodegen (31) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + WholeStageCodegen (32) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (28) + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #9 + InputAdapter + WholeStageCodegen (31) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 8eac0cc89246e..e6a4a0a87fd91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) +TakeOrderedAndProject (73) ++- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * BroadcastHashJoin Inner BuildRight (33) : : :- * Filter (16) : : : +- * HashAggregate (15) @@ -37,39 +37,41 @@ TakeOrderedAndProject (71) : : : +- * ColumnarToRow (21) : : : +- Scan parquet spark_catalog.default.store_sales (20) : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * Filter (39) - : : +- * ColumnarToRow (38) - : : +- Scan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) - +- BroadcastExchange (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * Filter (58) - : +- * ColumnarToRow (57) - : +- Scan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (70) + +- * Project (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (56) + : : +- * ColumnarToRow (55) + : : +- Scan parquet spark_catalog.default.customer (54) + : +- BroadcastExchange (60) + : +- * Filter (59) + : +- * ColumnarToRow (58) + : +- Scan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (63) (1) Scan parquet spark_catalog.default.customer @@ -115,7 +117,7 @@ Join condition: None Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(10) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#9, d_year#10] (11) BroadcastHashJoin [codegen id : 3] @@ -193,7 +195,7 @@ Join condition: None Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(26) ReusedExchange [Reuses operator id: 79] +(26) ReusedExchange [Reuses operator id: 81] Output [2]: [d_date_sk#24, d_year#25] (27) BroadcastHashJoin [codegen id : 6] @@ -277,7 +279,7 @@ Join condition: None Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(43) ReusedExchange [Reuses operator id: 75] +(43) ReusedExchange [Reuses operator id: 77] Output [2]: [d_date_sk#39, d_year#40] (44) BroadcastHashJoin [codegen id : 10] @@ -312,166 +314,174 @@ Results [2]: [c_customer_id#33 AS customer_id#44, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#44, year_total#45] Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) -(50) BroadcastExchange +(50) Project [codegen id : 11] +Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] Input [2]: [customer_id#44, year_total#45] + +(51) BroadcastExchange +Input [2]: [customer_id#46, year_total#47] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#44] +Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#44, year_total#45] +(53) Project [codegen id : 16] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -(53) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] +(54) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] +(55) ColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -(55) Filter [codegen id : 14] -Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(c_customer_id#47)) +(56) Filter [codegen id : 14] +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Condition : (isnotnull(c_customer_sk#48) AND isnotnull(c_customer_id#49)) -(56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(57) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(58) ColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -(58) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_customer_sk#50) +(59) Filter [codegen id : 12] +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_customer_sk#52) -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(60) BroadcastExchange +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#46] -Right keys [1]: [ws_bill_customer_sk#50] +(61) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#48] +Right keys [1]: [ws_bill_customer_sk#52] Join type: Inner Join condition: None -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, ws_sold_date_sk#52] -Input [7]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52] +(62) Project [codegen id : 14] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, ws_sold_date_sk#54] +Input [7]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#53, d_year#54] +(63) ReusedExchange [Reuses operator id: 81] +Output [2]: [d_date_sk#55, d_year#56] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#53] +(64) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join type: Inner Join condition: None -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, d_year#54] -Input [7]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54] - -(65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, d_year#54] -Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#51))] -Aggregate Attributes [1]: [sum#55] -Results [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56] - -(66) Exchange -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56] -Arguments: hashpartitioning(c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56] -Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_net_paid#51))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#51))#43] -Results [2]: [c_customer_id#47 AS customer_id#57, MakeDecimal(sum(UnscaledValue(ws_net_paid#51))#43,17,2) AS year_total#58] - -(68) BroadcastExchange -Input [2]: [customer_id#57, year_total#58] +(65) Project [codegen id : 14] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, d_year#56] +Input [7]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56] + +(66) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#53, d_year#56] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, sum#58] + +(67) Exchange +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(68) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56, sum#58] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#53))#43] +Results [2]: [c_customer_id#49 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#53))#43,17,2) AS year_total#60] + +(69) Project [codegen id : 15] +Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] +Input [2]: [customer_id#59, year_total#60] + +(70) BroadcastExchange +Input [2]: [customer_id#61, year_total#62] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#57] +Right keys [1]: [customer_id#61] Join type: Inner -Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#47 > 0.00) THEN (year_total#62 / year_total#47) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(70) Project [codegen id : 16] +(72) Project [codegen id : 16] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45, customer_id#57, year_total#58] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#61, year_total#62] -(71) TakeOrderedAndProject +(73) TakeOrderedAndProject Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (75) -+- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * Filter (76) + +- * ColumnarToRow (75) + +- Scan parquet spark_catalog.default.date_dim (74) -(72) Scan parquet spark_catalog.default.date_dim +(74) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(74) Filter [codegen id : 1] +(76) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(75) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (79) -+- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (81) ++- * Filter (80) + +- * ColumnarToRow (79) + +- Scan parquet spark_catalog.default.date_dim (78) -(76) Scan parquet spark_catalog.default.date_dim +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(77) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(78) Filter [codegen id : 1] +(80) Filter [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) -(79) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index e504fd8ce87c0..0281fc830b4be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -70,53 +70,55 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + Project [customer_id,year_total] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_sk,c_customer_id] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + Project [customer_id,year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + Filter [c_customer_sk,c_customer_id] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt index 528be1a965c23..0525697078939 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q18/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * HashAggregate (30) - +- Exchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) +TakeOrderedAndProject (29) ++- * HashAggregate (28) + +- Exchange (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) :- * Project (19) : +- * BroadcastHashJoin Inner BuildRight (18) : :- * Filter (3) @@ -24,12 +24,10 @@ TakeOrderedAndProject (31) : +- * Filter (9) : +- * ColumnarToRow (8) : +- Scan parquet spark_catalog.default.lineitem (7) - +- BroadcastExchange (25) - +- * BroadcastHashJoin LeftSemi BuildRight (24) - :- * Filter (22) - : +- * ColumnarToRow (21) - : +- Scan parquet spark_catalog.default.lineitem (20) - +- ReusedExchange (23) + +- BroadcastExchange (23) + +- * Filter (22) + +- * ColumnarToRow (21) + +- Scan parquet spark_catalog.default.lineitem (20) (1) Scan parquet spark_catalog.default.customer @@ -39,10 +37,10 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 5] Input [2]: [c_custkey#1, c_name#2] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 5] Input [2]: [c_custkey#1, c_name#2] Condition : isnotnull(c_custkey#1) @@ -50,7 +48,7 @@ Condition : isnotnull(c_custkey#1) Output [4]: [o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] Batched: true Location [not included in comparison]/{warehouse_dir}/orders] -PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] +PushedFilters: [IsNotNull(o_orderkey), IsNotNull(o_custkey)] ReadSchema: struct (5) ColumnarToRow [codegen id : 3] @@ -58,7 +56,7 @@ Input [4]: [o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] (6) Filter [codegen id : 3] Input [4]: [o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] -Condition : (isnotnull(o_custkey#4) AND isnotnull(o_orderkey#3)) +Condition : (isnotnull(o_orderkey#3) AND isnotnull(o_custkey#4)) (7) Scan parquet spark_catalog.default.lineitem Output [2]: [l_orderkey#7, l_quantity#8] @@ -114,13 +112,13 @@ Join condition: None Input [4]: [o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] Arguments: HashedRelationBroadcastMode(List(input[1, bigint, false]),false), [plan_id=3] -(18) BroadcastHashJoin [codegen id : 7] +(18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_custkey#1] Right keys [1]: [o_custkey#4] Join type: Inner Join condition: None -(19) Project [codegen id : 7] +(19) Project [codegen id : 5] Output [5]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6] Input [6]: [c_custkey#1, c_name#2, o_orderkey#3, o_custkey#4, o_totalprice#5, o_orderdate#6] @@ -131,55 +129,46 @@ Location [not included in comparison]/{warehouse_dir}/lineitem] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 6] +(21) ColumnarToRow [codegen id : 4] Input [2]: [l_orderkey#16, l_quantity#14] -(22) Filter [codegen id : 6] +(22) Filter [codegen id : 4] Input [2]: [l_orderkey#16, l_quantity#14] Condition : isnotnull(l_orderkey#16) -(23) ReusedExchange [Reuses operator id: 15] -Output [1]: [l_orderkey#7] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [l_orderkey#16] -Right keys [1]: [l_orderkey#7] -Join type: LeftSemi -Join condition: None - -(25) BroadcastExchange +(23) BroadcastExchange Input [2]: [l_orderkey#16, l_quantity#14] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=4] -(26) BroadcastHashJoin [codegen id : 7] +(24) BroadcastHashJoin [codegen id : 5] Left keys [1]: [o_orderkey#3] Right keys [1]: [l_orderkey#16] Join type: Inner Join condition: None -(27) Project [codegen id : 7] +(25) Project [codegen id : 5] Output [6]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6, l_quantity#14] Input [7]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6, l_orderkey#16, l_quantity#14] -(28) HashAggregate [codegen id : 7] +(26) HashAggregate [codegen id : 5] Input [6]: [c_custkey#1, c_name#2, o_orderkey#3, o_totalprice#5, o_orderdate#6, l_quantity#14] Keys [5]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5] Functions [1]: [partial_sum(l_quantity#14)] Aggregate Attributes [2]: [sum#17, isEmpty#18] Results [7]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum#19, isEmpty#20] -(29) Exchange +(27) Exchange Input [7]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum#19, isEmpty#20] Arguments: hashpartitioning(c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(30) HashAggregate [codegen id : 8] +(28) HashAggregate [codegen id : 6] Input [7]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum#19, isEmpty#20] Keys [5]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5] Functions [1]: [sum(l_quantity#14)] Aggregate Attributes [1]: [sum(l_quantity#14)#21] Results [6]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum(l_quantity#14)#21 AS sum(l_quantity)#22] -(31) TakeOrderedAndProject +(29) TakeOrderedAndProject Input [6]: [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum(l_quantity)#22] Arguments: 100, [o_totalprice#5 DESC NULLS LAST, o_orderdate#6 ASC NULLS FIRST], [c_name#2, c_custkey#1, o_orderkey#3, o_orderdate#6, o_totalprice#5, sum(l_quantity)#22] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt index dcc3345a18821..1db849362f164 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q18/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [o_totalprice,o_orderdate,c_name,c_custkey,o_orderkey,sum(l_quantity)] - WholeStageCodegen (8) + WholeStageCodegen (6) HashAggregate [c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice,sum,isEmpty] [sum(l_quantity),sum(l_quantity),sum,isEmpty] InputAdapter Exchange [c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice] #1 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice,l_quantity] [sum,isEmpty,sum,isEmpty] Project [c_custkey,c_name,o_orderkey,o_totalprice,o_orderdate,l_quantity] BroadcastHashJoin [o_orderkey,l_orderkey] @@ -17,7 +17,7 @@ TakeOrderedAndProject [o_totalprice,o_orderdate,c_name,c_custkey,o_orderkey,sum( BroadcastExchange #2 WholeStageCodegen (3) BroadcastHashJoin [o_orderkey,l_orderkey] - Filter [o_custkey,o_orderkey] + Filter [o_orderkey,o_custkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.orders [o_orderkey,o_custkey,o_totalprice,o_orderdate] @@ -37,11 +37,8 @@ TakeOrderedAndProject [o_totalprice,o_orderdate,c_name,c_custkey,o_orderkey,sum( Scan parquet spark_catalog.default.lineitem [l_orderkey,l_quantity] InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) - BroadcastHashJoin [l_orderkey,l_orderkey] - Filter [l_orderkey] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.lineitem [l_orderkey,l_quantity] - InputAdapter - ReusedExchange [l_orderkey] #3 + WholeStageCodegen (4) + Filter [l_orderkey] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.lineitem [l_orderkey,l_quantity] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt index 1f8efaaf1a3a0..558e9adab6196 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q19/explain.txt @@ -36,7 +36,7 @@ Input [6]: [l_partkey#1, l_quantity#2, l_extendedprice#3, l_discount#4, l_shipin Output [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] Batched: true Location [not included in comparison]/{warehouse_dir}/part] -PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#11),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#12),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#13),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] +PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#11),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#6),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#12),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] (7) Filter [codegen id : 1] Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] -Condition : (((isnotnull(p_size#9) AND (p_size#9 >= 1)) AND isnotnull(p_partkey#7)) AND (((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#9 <= 5)) OR (((p_brand#8 = Brand#12) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#9 <= 10))) OR (((p_brand#8 = Brand#13) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#9 <= 15)))) +Condition : (((isnotnull(p_size#9) AND (p_size#9 >= 1)) AND isnotnull(p_partkey#7)) AND (((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#9 <= 5)) OR (((p_brand#8 = Brand#6) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#9 <= 10))) OR (((p_brand#8 = Brand#12) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#9 <= 15)))) (8) BroadcastExchange Input [4]: [p_partkey#7, p_brand#8, p_size#9, p_container#10] @@ -54,7 +54,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [pl Left keys [1]: [l_partkey#1] Right keys [1]: [p_partkey#7] Join type: Inner -Join condition: (((((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#2 >= 1)) AND (l_quantity#2 <= 11)) AND (p_size#9 <= 5)) OR (((((p_brand#8 = Brand#12) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#2 >= 10)) AND (l_quantity#2 <= 20)) AND (p_size#9 <= 10))) OR (((((p_brand#8 = Brand#13) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#2 >= 20)) AND (l_quantity#2 <= 30)) AND (p_size#9 <= 15))) +Join condition: (((((((p_brand#8 = Brand#11) AND p_container#10 IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#2 >= 1)) AND (l_quantity#2 <= 11)) AND (p_size#9 <= 5)) OR (((((p_brand#8 = Brand#6) AND p_container#10 IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#2 >= 10)) AND (l_quantity#2 <= 20)) AND (p_size#9 <= 10))) OR (((((p_brand#8 = Brand#12) AND p_container#10 IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#2 >= 20)) AND (l_quantity#2 <= 30)) AND (p_size#9 <= 15))) (10) Project [codegen id : 2] Output [2]: [l_extendedprice#3, l_discount#4] @@ -64,17 +64,17 @@ Input [8]: [l_partkey#1, l_quantity#2, l_extendedprice#3, l_discount#4, p_partke Input [2]: [l_extendedprice#3, l_discount#4] Keys: [] Functions [1]: [partial_sum((l_extendedprice#3 * (1 - l_discount#4)))] -Aggregate Attributes [2]: [sum#14, isEmpty#15] -Results [2]: [sum#16, isEmpty#17] +Aggregate Attributes [2]: [sum#13, isEmpty#14] +Results [2]: [sum#15, isEmpty#16] (12) Exchange -Input [2]: [sum#16, isEmpty#17] +Input [2]: [sum#15, isEmpty#16] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] (13) HashAggregate [codegen id : 3] -Input [2]: [sum#16, isEmpty#17] +Input [2]: [sum#15, isEmpty#16] Keys: [] Functions [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))] -Aggregate Attributes [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#18] -Results [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#18 AS revenue#19] +Aggregate Attributes [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#17] +Results [1]: [sum((l_extendedprice#3 * (1 - l_discount#4)))#17 AS revenue#18] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt index 5f6ccb95b376a..837b3956ef61c 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == -* Sort (37) -+- Exchange (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (28) - : +- * BroadcastHashJoin LeftSemi BuildRight (27) +* Sort (35) ++- Exchange (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (26) + : +- * BroadcastHashJoin LeftSemi BuildRight (25) : :- * Filter (3) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.supplier (1) - : +- BroadcastExchange (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) + : +- BroadcastExchange (24) + : +- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) : :- * BroadcastHashJoin LeftSemi BuildRight (12) : : :- * Filter (6) : : : +- * ColumnarToRow (5) @@ -20,51 +20,49 @@ : : +- * Filter (9) : : +- * ColumnarToRow (8) : : +- Scan parquet spark_catalog.default.part (7) - : +- BroadcastExchange (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * BroadcastHashJoin LeftSemi BuildRight (18) - : :- * Project (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet spark_catalog.default.lineitem (13) - : +- ReusedExchange (17) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet spark_catalog.default.nation (29) + : +- BroadcastExchange (21) + : +- * Filter (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * Filter (15) + : +- * ColumnarToRow (14) + : +- Scan parquet spark_catalog.default.lineitem (13) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- * ColumnarToRow (28) + +- Scan parquet spark_catalog.default.nation (27) (1) Scan parquet spark_catalog.default.supplier Output [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] Batched: true Location [not included in comparison]/{warehouse_dir}/supplier] -PushedFilters: [IsNotNull(s_nationkey), IsNotNull(s_suppkey)] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 6] Input [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 6] Input [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] -Condition : (isnotnull(s_nationkey#4) AND isnotnull(s_suppkey#1)) +Condition : (isnotnull(s_suppkey#1) AND isnotnull(s_nationkey#4)) (4) Scan parquet spark_catalog.default.partsupp Output [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] Batched: true Location [not included in comparison]/{warehouse_dir}/partsupp] -PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] +PushedFilters: [IsNotNull(ps_partkey), IsNotNull(ps_availqty), IsNotNull(ps_suppkey)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 5] +(5) ColumnarToRow [codegen id : 4] Input [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] -(6) Filter [codegen id : 5] +(6) Filter [codegen id : 4] Input [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] -Condition : ((isnotnull(ps_availqty#7) AND isnotnull(ps_partkey#5)) AND isnotnull(ps_suppkey#6)) +Condition : ((isnotnull(ps_partkey#5) AND isnotnull(ps_availqty#7)) AND isnotnull(ps_suppkey#6)) (7) Scan parquet spark_catalog.default.part Output [2]: [p_partkey#8, p_name#9] @@ -88,7 +86,7 @@ Input [2]: [p_partkey#8, p_name#9] Input [1]: [p_partkey#8] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ps_partkey#5] Right keys [1]: [p_partkey#8] Join type: LeftSemi @@ -101,113 +99,104 @@ Location [not included in comparison]/{warehouse_dir}/lineitem] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 2] Input [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 2] Input [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] Condition : ((((isnotnull(l_shipdate#13) AND (l_shipdate#13 >= 1994-01-01)) AND (l_shipdate#13 < 1995-01-01)) AND isnotnull(l_partkey#10)) AND isnotnull(l_suppkey#11)) -(16) Project [codegen id : 3] +(16) Project [codegen id : 2] Output [3]: [l_partkey#10, l_suppkey#11, l_quantity#12] Input [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] -(17) ReusedExchange [Reuses operator id: 11] -Output [1]: [p_partkey#8] - -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [l_partkey#10] -Right keys [1]: [p_partkey#8] -Join type: LeftSemi -Join condition: None - -(19) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 2] Input [3]: [l_partkey#10, l_suppkey#11, l_quantity#12] Keys [2]: [l_partkey#10, l_suppkey#11] Functions [1]: [partial_sum(l_quantity#12)] Aggregate Attributes [2]: [sum#14, isEmpty#15] Results [4]: [l_partkey#10, l_suppkey#11, sum#16, isEmpty#17] -(20) Exchange +(18) Exchange Input [4]: [l_partkey#10, l_suppkey#11, sum#16, isEmpty#17] Arguments: hashpartitioning(l_partkey#10, l_suppkey#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 3] Input [4]: [l_partkey#10, l_suppkey#11, sum#16, isEmpty#17] Keys [2]: [l_partkey#10, l_suppkey#11] Functions [1]: [sum(l_quantity#12)] Aggregate Attributes [1]: [sum(l_quantity#12)#18] Results [3]: [(0.5 * sum(l_quantity#12)#18) AS (0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] -(22) Filter [codegen id : 4] +(20) Filter [codegen id : 3] Input [3]: [(0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] Condition : isnotnull((0.5 * sum(l_quantity))#19) -(23) BroadcastExchange +(21) BroadcastExchange Input [3]: [(0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true], input[2, bigint, true]),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 4] Left keys [2]: [ps_partkey#5, ps_suppkey#6] Right keys [2]: [l_partkey#10, l_suppkey#11] Join type: Inner Join condition: (cast(ps_availqty#7 as decimal(22,1)) > (0.5 * sum(l_quantity))#19) -(25) Project [codegen id : 5] +(23) Project [codegen id : 4] Output [1]: [ps_suppkey#6] Input [6]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7, (0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] -(26) BroadcastExchange +(24) BroadcastExchange Input [1]: [ps_suppkey#6] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 7] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_suppkey#1] Right keys [1]: [ps_suppkey#6] Join type: LeftSemi Join condition: None -(28) Project [codegen id : 7] +(26) Project [codegen id : 6] Output [3]: [s_name#2, s_address#3, s_nationkey#4] Input [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] -(29) Scan parquet spark_catalog.default.nation +(27) Scan parquet spark_catalog.default.nation Output [2]: [n_nationkey#20, n_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/nation] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 6] +(28) ColumnarToRow [codegen id : 5] Input [2]: [n_nationkey#20, n_name#21] -(31) Filter [codegen id : 6] +(29) Filter [codegen id : 5] Input [2]: [n_nationkey#20, n_name#21] Condition : ((isnotnull(n_name#21) AND (n_name#21 = CANADA)) AND isnotnull(n_nationkey#20)) -(32) Project [codegen id : 6] +(30) Project [codegen id : 5] Output [1]: [n_nationkey#20] Input [2]: [n_nationkey#20, n_name#21] -(33) BroadcastExchange +(31) BroadcastExchange Input [1]: [n_nationkey#20] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_nationkey#4] Right keys [1]: [n_nationkey#20] Join type: Inner Join condition: None -(35) Project [codegen id : 7] +(33) Project [codegen id : 6] Output [2]: [s_name#2, s_address#3] Input [4]: [s_name#2, s_address#3, s_nationkey#4, n_nationkey#20] -(36) Exchange +(34) Exchange Input [2]: [s_name#2, s_address#3] Arguments: rangepartitioning(s_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) Sort [codegen id : 8] +(35) Sort [codegen id : 7] Input [2]: [s_name#2, s_address#3] Arguments: [s_name#2 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt index 6cc10adcfb20c..65abd21e08684 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (8) +WholeStageCodegen (7) Sort [s_name] InputAdapter Exchange [s_name] #1 - WholeStageCodegen (7) + WholeStageCodegen (6) Project [s_name,s_address] BroadcastHashJoin [s_nationkey,n_nationkey] Project [s_name,s_address,s_nationkey] BroadcastHashJoin [s_suppkey,ps_suppkey] - Filter [s_nationkey,s_suppkey] + Filter [s_suppkey,s_nationkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.supplier [s_suppkey,s_name,s_address,s_nationkey] InputAdapter BroadcastExchange #2 - WholeStageCodegen (5) + WholeStageCodegen (4) Project [ps_suppkey] BroadcastHashJoin [ps_partkey,ps_suppkey,l_partkey,l_suppkey,ps_availqty,(0.5 * sum(l_quantity))] BroadcastHashJoin [ps_partkey,p_partkey] - Filter [ps_availqty,ps_partkey,ps_suppkey] + Filter [ps_partkey,ps_availqty,ps_suppkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.partsupp [ps_partkey,ps_suppkey,ps_availqty] @@ -31,24 +31,21 @@ WholeStageCodegen (8) Scan parquet spark_catalog.default.part [p_partkey,p_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [(0.5 * sum(l_quantity))] HashAggregate [l_partkey,l_suppkey,sum,isEmpty] [sum(l_quantity),(0.5 * sum(l_quantity)),sum,isEmpty] InputAdapter Exchange [l_partkey,l_suppkey] #5 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [l_partkey,l_suppkey,l_quantity] [sum,isEmpty,sum,isEmpty] - BroadcastHashJoin [l_partkey,p_partkey] - Project [l_partkey,l_suppkey,l_quantity] - Filter [l_shipdate,l_partkey,l_suppkey] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.lineitem [l_partkey,l_suppkey,l_quantity,l_shipdate] - InputAdapter - ReusedExchange [p_partkey] #3 + Project [l_partkey,l_suppkey,l_quantity] + Filter [l_shipdate,l_partkey,l_suppkey] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.lineitem [l_partkey,l_suppkey,l_quantity,l_shipdate] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (5) Project [n_nationkey] Filter [n_name,n_nationkey] ColumnarToRow diff --git a/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt index d9203026e7da7..29bde9b08088d 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q21/explain.txt @@ -58,7 +58,7 @@ Condition : (isnotnull(s_suppkey#1) AND isnotnull(s_nationkey#3)) Output [4]: [l_orderkey#4, l_suppkey#5, l_commitdate#6, l_receiptdate#7] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] -PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] +PushedFilters: [IsNotNull(l_receiptdate), IsNotNull(l_commitdate), IsNotNull(l_orderkey), IsNotNull(l_suppkey)] ReadSchema: struct (5) ColumnarToRow [codegen id : 3] @@ -66,7 +66,7 @@ Input [4]: [l_orderkey#4, l_suppkey#5, l_commitdate#6, l_receiptdate#7] (6) Filter [codegen id : 3] Input [4]: [l_orderkey#4, l_suppkey#5, l_commitdate#6, l_receiptdate#7] -Condition : ((((isnotnull(l_receiptdate#7) AND isnotnull(l_commitdate#6)) AND (l_receiptdate#7 > l_commitdate#6)) AND isnotnull(l_suppkey#5)) AND isnotnull(l_orderkey#4)) +Condition : ((((isnotnull(l_receiptdate#7) AND isnotnull(l_commitdate#6)) AND (l_receiptdate#7 > l_commitdate#6)) AND isnotnull(l_orderkey#4)) AND isnotnull(l_suppkey#5)) (7) Project [codegen id : 3] Output [2]: [l_orderkey#4, l_suppkey#5] diff --git a/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt index 4687bf94b96cf..3e38ff7c5e155 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q21/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [numwait,s_name] BroadcastHashJoin [l_orderkey,l_orderkey,l_suppkey,l_suppkey] BroadcastHashJoin [l_orderkey,l_orderkey,l_suppkey,l_suppkey] Project [l_orderkey,l_suppkey] - Filter [l_receiptdate,l_commitdate,l_suppkey,l_orderkey] + Filter [l_receiptdate,l_commitdate,l_orderkey,l_suppkey] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.lineitem [l_orderkey,l_suppkey,l_commitdate,l_receiptdate] From c591da36918ec8cf69351b091b8783b87eb3ed2c Mon Sep 17 00:00:00 2001 From: Kapil Kumar Singh Date: Wed, 22 Mar 2023 12:37:28 +0530 Subject: [PATCH 5/6] Fix tests --- .../sql/catalyst/optimizer/Optimizer.scala | 4 + .../approved-plans-v1_4/q13.sf100/explain.txt | 405 +++++++++++------- .../q13.sf100/simplified.txt | 129 +++--- .../approved-plans-v1_4/q13/explain.txt | 200 ++++----- .../approved-plans-v1_4/q13/simplified.txt | 50 +-- .../org/apache/spark/sql/JoinSuite.scala | 4 +- .../org/apache/spark/sql/SubquerySuite.scala | 10 +- .../datasources/SchemaPruningSuite.scala | 8 +- 8 files changed, 473 insertions(+), 337 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f4d4103b8f5f9..6a9e7d5b7935b 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 @@ -176,6 +176,10 @@ abstract class Optimizer(catalogManager: CatalogManager) // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). Batch("Subquery", FixedPoint(1), OptimizeSubqueries) :: + Batch("Preparation for RewriteSubquery", fixedPoint, + // Boolean simplification is done before RewritePredicateSubquery so that predicates + // containing IN and EXISTS are simplified before rewriting. Eg. NOT(NOT IN) = IN + BooleanSimplification) :: Batch("RewriteSubquery", Once, RewritePredicateSubquery) :: Batch("Replace Operators", fixedPoint, diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 5dc4c4496c8b0..8a6230793b12a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -1,38 +1,41 @@ == Physical Plan == -* HashAggregate (34) -+- Exchange (33) - +- * HashAggregate (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.customer_demographics (4) - : : : +- BroadcastExchange (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.household_demographics (10) - : : +- ReusedExchange (16) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet spark_catalog.default.store (19) - +- BroadcastExchange (29) - +- * Project (28) - +- * Filter (27) - +- * ColumnarToRow (26) - +- Scan parquet spark_catalog.default.customer_address (25) +* HashAggregate (37) ++- Exchange (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (18) + : : : +- * SortMergeJoin Inner (17) + : : : :- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet spark_catalog.default.store (4) + : : : +- * Sort (16) + : : : +- Exchange (15) + : : : +- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet spark_catalog.default.customer_demographics (12) + : : +- BroadcastExchange (22) + : : +- * Filter (21) + : : +- * ColumnarToRow (20) + : : +- Scan parquet spark_catalog.default.household_demographics (19) + : +- BroadcastExchange (29) + : +- * Project (28) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet spark_catalog.default.customer_address (25) + +- ReusedExchange (32) (1) Scan parquet spark_catalog.default.store_sales @@ -40,193 +43,297 @@ Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quan Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 2] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 2] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) +Condition : ((((((isnotnull(ss_store_sk#4) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND might_contain(Subquery scalar-subquery#12, [id=#13], xxhash64(ss_hdemo_sk#2, 42))) AND might_contain(Subquery scalar-subquery#14, [id=#15], xxhash64(ss_addr_sk#3, 42))) -(4) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] +(4) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#16] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), 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}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] +Input [1]: [s_store_sk#16] (6) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] -Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College ))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )))) +Input [1]: [s_store_sk#16] +Condition : isnotnull(s_store_sk#16) (7) BroadcastExchange -Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] +Input [1]: [s_store_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#12] +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#16] Join type: Inner -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: None -(9) Project [codegen id : 6] -Output [11]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14] -Input [13]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] +(9) Project [codegen id : 2] +Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#16] -(10) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#15, hd_dep_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct +(10) Exchange +Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Arguments: hashpartitioning(ss_cdemo_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) ColumnarToRow [codegen id : 2] -Input [2]: [hd_demo_sk#15, hd_dep_count#16] +(11) Sort [codegen id : 3] +Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Arguments: [ss_cdemo_sk#1 ASC NULLS FIRST], false, 0 -(12) Filter [codegen id : 2] -Input [2]: [hd_demo_sk#15, hd_dep_count#16] -Condition : (isnotnull(hd_demo_sk#15) AND ((hd_dep_count#16 = 3) OR (hd_dep_count#16 = 1))) +(12) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(13) BroadcastExchange -Input [2]: [hd_demo_sk#15, hd_dep_count#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(13) ColumnarToRow [codegen id : 4] +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#15] -Join type: Inner -Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#16 = 1))) +(14) Filter [codegen id : 4] +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Condition : isnotnull(cd_demo_sk#17) -(15) Project [codegen id : 6] -Output [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Input [13]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14, hd_demo_sk#15, hd_dep_count#16] +(15) Exchange +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Arguments: hashpartitioning(cd_demo_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(16) ReusedExchange [Reuses operator id: 39] -Output [1]: [d_date_sk#17] +(16) Sort [codegen id : 5] +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Arguments: [cd_demo_sk#17 ASC NULLS FIRST], false, 0 -(17) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#17] +(17) SortMergeJoin [codegen id : 9] +Left keys [1]: [ss_cdemo_sk#1] +Right keys [1]: [cd_demo_sk#17] Join type: Inner -Join condition: None +Join condition: ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) -(18) Project [codegen id : 6] -Output [6]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9] -Input [8]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, d_date_sk#17] +(18) Project [codegen id : 9] +Output [10]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#18, cd_education_status#19] +Input [12]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -(19) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] +(19) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#20, hd_dep_count#21] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] +ReadSchema: struct -(20) ColumnarToRow [codegen id : 4] -Input [1]: [s_store_sk#18] +(20) ColumnarToRow [codegen id : 6] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] -(21) Filter [codegen id : 4] -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) +(21) Filter [codegen id : 6] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] +Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1))) (22) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#18] +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#20] Join type: Inner -Join condition: None +Join condition: (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))) -(24) Project [codegen id : 6] -Output [5]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9] -Input [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, s_store_sk#18] +(24) Project [codegen id : 9] +Output [6]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Input [12]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21] (25) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#19, ca_state#20, ca_country#21] +Output [3]: [ca_address_sk#22, ca_state#23, ca_country#24] 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, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] ReadSchema: struct -(26) ColumnarToRow [codegen id : 5] -Input [3]: [ca_address_sk#19, ca_state#20, ca_country#21] +(26) ColumnarToRow [codegen id : 7] +Input [3]: [ca_address_sk#22, ca_state#23, ca_country#24] -(27) Filter [codegen id : 5] -Input [3]: [ca_address_sk#19, ca_state#20, ca_country#21] -Condition : (((isnotnull(ca_country#21) AND (ca_country#21 = United States)) AND isnotnull(ca_address_sk#19)) AND ((ca_state#20 IN (TX,OH) OR ca_state#20 IN (OR,NM,KY)) OR ca_state#20 IN (VA,TX,MS))) +(27) Filter [codegen id : 7] +Input [3]: [ca_address_sk#22, ca_state#23, ca_country#24] +Condition : (((isnotnull(ca_country#24) AND (ca_country#24 = United States)) AND isnotnull(ca_address_sk#22)) AND ((ca_state#23 IN (TX,OH) OR ca_state#23 IN (OR,NM,KY)) OR ca_state#23 IN (VA,TX,MS))) -(28) Project [codegen id : 5] -Output [2]: [ca_address_sk#19, ca_state#20] -Input [3]: [ca_address_sk#19, ca_state#20, ca_country#21] +(28) Project [codegen id : 7] +Output [2]: [ca_address_sk#22, ca_state#23] +Input [3]: [ca_address_sk#22, ca_state#23, ca_country#24] (29) BroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#22] Join type: Inner -Join condition: ((((ca_state#20 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#20 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#20 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) +Join condition: ((((ca_state#23 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#23 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#23 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) + +(31) Project [codegen id : 9] +Output [4]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +Input [8]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#22, ca_state#23] + +(32) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#25] -(31) Project [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#10] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 9] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [7]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ca_address_sk#19, ca_state#20] +Input [5]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#25] -(32) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 9] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Keys: [] Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28] -Results [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +Aggregate Attributes [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] +Results [7]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39] -(33) Exchange -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(36) Exchange +Input [7]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(34) HashAggregate [codegen id : 7] -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +(37) HashAggregate [codegen id : 10] +Input [7]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#36, avg(UnscaledValue(ss_ext_sales_price#7))#37, avg(UnscaledValue(ss_ext_wholesale_cost#8))#38, sum(UnscaledValue(ss_ext_wholesale_cost#8))#39] -Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledValue(ss_ext_sales_price#7))#37 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#41, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#42, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#39,17,2) AS sum(ss_ext_wholesale_cost)#43] +Aggregate Attributes [4]: [avg(ss_quantity#5)#40, avg(UnscaledValue(ss_ext_sales_price#7))#41, avg(UnscaledValue(ss_ext_wholesale_cost#8))#42, sum(UnscaledValue(ss_ext_wholesale_cost#8))#43] +Results [4]: [avg(ss_quantity#5)#40 AS avg(ss_quantity)#44, cast((avg(UnscaledValue(ss_ext_sales_price#7))#41 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#45, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#42 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#46, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#43,17,2) AS sum(ss_ext_wholesale_cost)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * Project (38) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet spark_catalog.default.date_dim (35) +Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#12, [id=#13] +ObjectHashAggregate (44) ++- Exchange (43) + +- ObjectHashAggregate (42) + +- * Project (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet spark_catalog.default.household_demographics (38) + + +(38) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#20, hd_dep_count#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), Or(Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1)),EqualTo(hd_dep_count,1))] +ReadSchema: struct + +(39) ColumnarToRow [codegen id : 1] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] + +(40) Filter [codegen id : 1] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] +Condition : (isnotnull(hd_demo_sk#20) AND (((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1)) OR (hd_dep_count#21 = 1))) + +(41) Project [codegen id : 1] +Output [1]: [hd_demo_sk#20] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] + +(42) ObjectHashAggregate +Input [1]: [hd_demo_sk#20] +Keys: [] +Functions [1]: [partial_bloom_filter_agg(xxhash64(hd_demo_sk#20, 42), 1952, 45226, 0, 0)] +Aggregate Attributes [1]: [buf#48] +Results [1]: [buf#49] + +(43) Exchange +Input [1]: [buf#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(44) ObjectHashAggregate +Input [1]: [buf#49] +Keys: [] +Functions [1]: [bloom_filter_agg(xxhash64(hd_demo_sk#20, 42), 1952, 45226, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(hd_demo_sk#20, 42), 1952, 45226, 0, 0)#50] +Results [1]: [bloom_filter_agg(xxhash64(hd_demo_sk#20, 42), 1952, 45226, 0, 0)#50 AS bloomFilter#51] + +Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#14, [id=#15] +ObjectHashAggregate (51) ++- Exchange (50) + +- ObjectHashAggregate (49) + +- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet spark_catalog.default.customer_address (45) + + +(45) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#22, ca_state#23, ca_country#24] +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, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 1] +Input [3]: [ca_address_sk#22, ca_state#23, ca_country#24] + +(47) Filter [codegen id : 1] +Input [3]: [ca_address_sk#22, ca_state#23, ca_country#24] +Condition : (((isnotnull(ca_country#24) AND (ca_country#24 = United States)) AND isnotnull(ca_address_sk#22)) AND ((ca_state#23 IN (TX,OH) OR ca_state#23 IN (OR,NM,KY)) OR ca_state#23 IN (VA,TX,MS))) + +(48) Project [codegen id : 1] +Output [1]: [ca_address_sk#22] +Input [3]: [ca_address_sk#22, ca_state#23, ca_country#24] + +(49) ObjectHashAggregate +Input [1]: [ca_address_sk#22] +Keys: [] +Functions [1]: [partial_bloom_filter_agg(xxhash64(ca_address_sk#22, 42), 136816, 1959741, 0, 0)] +Aggregate Attributes [1]: [buf#52] +Results [1]: [buf#53] + +(50) Exchange +Input [1]: [buf#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] + +(51) ObjectHashAggregate +Input [1]: [buf#53] +Keys: [] +Functions [1]: [bloom_filter_agg(xxhash64(ca_address_sk#22, 42), 136816, 1959741, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(ca_address_sk#22, 42), 136816, 1959741, 0, 0)#54] +Results [1]: [bloom_filter_agg(xxhash64(ca_address_sk#22, 42), 136816, 1959741, 0, 0)#54 AS bloomFilter#55] + +Subquery:3 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 +BroadcastExchange (56) ++- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet spark_catalog.default.date_dim (52) -(35) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#44] +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#56] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#17, d_year#44] +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#25, d_year#56] -(37) Filter [codegen id : 1] -Input [2]: [d_date_sk#17, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#17)) +(54) Filter [codegen id : 1] +Input [2]: [d_date_sk#25, d_year#56] +Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2001)) AND isnotnull(d_date_sk#25)) -(38) Project [codegen id : 1] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_year#44] +(55) Project [codegen id : 1] +Output [1]: [d_date_sk#25] +Input [2]: [d_date_sk#25, d_year#56] -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(56) BroadcastExchange +Input [1]: [d_date_sk#25] +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/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index 095faf34c3866..748d159a42601 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt @@ -1,59 +1,88 @@ -WholeStageCodegen (7) +WholeStageCodegen (10) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] InputAdapter Exchange #1 - WholeStageCodegen (6) + WholeStageCodegen (9) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Filter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - 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 #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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,cd_marital_status,cd_education_status] + SortMergeJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + InputAdapter + WholeStageCodegen (3) + Sort [ss_cdemo_sk] + InputAdapter + Exchange [ss_cdemo_sk] #2 + WholeStageCodegen (2) + Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_net_profit] + Subquery #2 + ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(hd_demo_sk, 42), 1952, 45226, 0, 0),bloomFilter,buf] + Exchange #4 + ObjectHashAggregate [hd_demo_sk] [buf,buf] + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_demo_sk,hd_dep_count] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + Subquery #3 + ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(ca_address_sk, 42), 136816, 1959741, 0, 0),bloomFilter,buf] + Exchange #5 + 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.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + 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 #6 + WholeStageCodegen (1) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [hd_demo_sk,hd_dep_count] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + WholeStageCodegen (5) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #7 + WholeStageCodegen (4) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (6) + Filter [hd_demo_sk,hd_dep_count] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store [s_store_sk] + BroadcastExchange #9 + WholeStageCodegen (7) + 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 #6 - WholeStageCodegen (5) - 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] + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 21240628e9ff6..f87a6247f835a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -4,12 +4,12 @@ +- * HashAggregate (32) +- * Project (31) +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) : : : :- * Project (9) : : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : : :- * Filter (3) @@ -19,20 +19,20 @@ : : : : +- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (23) - : +- * Filter (22) - : +- * ColumnarToRow (21) - : +- Scan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet spark_catalog.default.household_demographics (26) + : : : +- BroadcastExchange (13) + : : : +- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet spark_catalog.default.customer_demographics (10) + : : +- BroadcastExchange (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet spark_catalog.default.household_demographics (16) + : +- BroadcastExchange (26) + : +- * Project (25) + : +- * Filter (24) + : +- * ColumnarToRow (23) + : +- Scan parquet spark_catalog.default.customer_address (22) + +- ReusedExchange (29) (1) Scan parquet spark_catalog.default.store_sales @@ -40,7 +40,7 @@ Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quan Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00)))] ReadSchema: struct (2) ColumnarToRow [codegen id : 6] @@ -48,7 +48,7 @@ Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quant (3) Filter [codegen id : 6] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) +Condition : ((((isnotnull(ss_store_sk#4) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) (4) Scan parquet spark_catalog.default.store Output [1]: [s_store_sk#12] @@ -78,106 +78,106 @@ Join condition: None Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -(10) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +(10) 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}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct (11) ColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Input [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] (12) Filter [codegen id : 2] -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (TX,OH) OR ca_state#14 IN (OR,NM,KY)) OR ca_state#14 IN (VA,TX,MS))) - -(13) Project [codegen id : 2] -Output [2]: [ca_address_sk#13, ca_state#14] -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] - -(14) BroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) - -(16) Project [codegen id : 6] -Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14] +Input [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] +Condition : isnotnull(cd_demo_sk#13) -(17) ReusedExchange [Reuses operator id: 39] -Output [1]: [d_date_sk#16] +(13) BroadcastExchange +Input [3]: [cd_demo_sk#13, cd_marital_status#14, cd_education_status#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#16] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_cdemo_sk#1] +Right keys [1]: [cd_demo_sk#13] Join type: Inner -Join condition: None +Join condition: ((((((cd_marital_status#14 = M) AND (cd_education_status#15 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#14 = S) AND (cd_education_status#15 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#14 = W) AND (cd_education_status#15 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) -(19) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#16] +(15) Project [codegen id : 6] +Output [10]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#14, cd_education_status#15] +Input [12]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, 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#17, cd_marital_status#18, cd_education_status#19] +(16) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#16, hd_dep_count#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), 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}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] +ReadSchema: struct -(21) ColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(17) ColumnarToRow [codegen id : 3] +Input [2]: [hd_demo_sk#16, hd_dep_count#17] -(22) Filter [codegen id : 4] -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Condition : (isnotnull(cd_demo_sk#17) AND ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) OR ((cd_marital_status#18 = S) AND (cd_education_status#19 = College ))) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )))) +(18) Filter [codegen id : 3] +Input [2]: [hd_demo_sk#16, hd_dep_count#17] +Condition : (isnotnull(hd_demo_sk#16) AND ((hd_dep_count#17 = 3) OR (hd_dep_count#17 = 1))) -(23) BroadcastExchange -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(19) BroadcastExchange +Input [2]: [hd_demo_sk#16, hd_dep_count#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#17] +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#16] Join type: Inner -Join condition: ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: (((((((cd_marital_status#14 = M) AND (cd_education_status#15 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#14 = S) AND (cd_education_status#15 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#14 = W) AND (cd_education_status#15 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) -(25) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19] -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(21) Project [codegen id : 6] +Output [6]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Input [12]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#14, cd_education_status#15, hd_demo_sk#16, hd_dep_count#17] -(26) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_dep_count#21] +(22) 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}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -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, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] +ReadSchema: struct -(27) ColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#20, hd_dep_count#21] +(23) ColumnarToRow [codegen id : 4] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] -(28) Filter [codegen id : 5] -Input [2]: [hd_demo_sk#20, hd_dep_count#21] -Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1))) +(24) 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 (TX,OH) OR ca_state#19 IN (OR,NM,KY)) OR ca_state#19 IN (VA,TX,MS))) -(29) BroadcastExchange -Input [2]: [hd_demo_sk#20, hd_dep_count#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(25) 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] + +(26) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: ((((ca_state#19 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#19 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#19 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) + +(28) Project [codegen id : 6] +Output [4]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +Input [8]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#18, ca_state#19] + +(29) ReusedExchange [Reuses operator id: 39] +Output [1]: [d_date_sk#21] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#20] +Left keys [1]: [ss_sold_date_sk#10] +Right keys [1]: [d_date_sk#21] Join type: Inner -Join condition: (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))) +Join condition: None (31) Project [codegen id : 6] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21] +Input [5]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#21] (32) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] @@ -208,25 +208,25 @@ BroadcastExchange (39) (35) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#44] +Output [2]: [d_date_sk#21, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#16, d_year#44] +Input [2]: [d_date_sk#21, d_year#44] (37) Filter [codegen id : 1] -Input [2]: [d_date_sk#16, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#21, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#21)) (38) Project [codegen id : 1] -Output [1]: [d_date_sk#16] -Input [2]: [d_date_sk#16, d_year#44] +Output [1]: [d_date_sk#21] +Input [2]: [d_date_sk#21, d_year#44] (39) BroadcastExchange -Input [1]: [d_date_sk#16] +Input [1]: [d_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index a3e168ff37bfb..a6bc26aff8fdc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (7) WholeStageCodegen (6) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + Filter [ss_store_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_net_profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] @@ -36,24 +36,24 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - 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] + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #5 + WholeStageCodegen (3) + Filter [hd_demo_sk,hd_dep_count] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - 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] + 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 #6 - WholeStageCodegen (5) - Filter [hd_demo_sk,hd_dep_count] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + ReusedExchange [d_date_sk] #2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 6dd34d41cf6c1..2518cc2a63955 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1158,12 +1158,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan var joinExec = assertJoin(( "select * from testData where key not in (select a from testData2)", classOf[BroadcastHashJoinExec])) - assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) + assert(!joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) // negative not in subquery case since multi-column is not supported assertJoin(( "select * from testData where (key, key + 1) not in (select * from testData2)", - classOf[BroadcastNestedLoopJoinExec])) + classOf[BroadcastHashJoinExec])) // positive hand-written left anti join // testData.key nullable false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 4d76013d65935..86fe34ba3b549 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2004,13 +2004,9 @@ class SubquerySuite extends QueryTest df = sql("select * from l where b = 5.0 and a not in (select c from r where d = b + 10)") checkAnswer(df, Row(null, 5.0) :: Nil) - if (enableNAAJ) { - joinExec = findJoinExec(df) - assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) - assert(joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) - } else { - assert(findJoinExec(df).isInstanceOf[BroadcastNestedLoopJoinExec]) - } + joinExec = findJoinExec(df) + assert(joinExec.isInstanceOf[BroadcastHashJoinExec]) + assert(!joinExec.asInstanceOf[BroadcastHashJoinExec].isNullAwareAntiJoin) // multi column not in subquery df = sql("select * from l where (a, b) not in (select c, d from r where c > 10)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index bd9c79e5b9648..baa6a6fc8719d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -658,10 +658,10 @@ abstract class SchemaPruningSuite | and e.employer.name = c.employer.company.name) |""".stripMargin) checkScan(query, - "struct," + - "employer:struct>>", - "struct," + - "employer:struct>") + "struct," + + "employer:struct>>", + "struct," + + "employer:struct>") checkAnswer(query, Row(3)) } } From b1ed7bee8b91faf8286963c7867eeed9f781b487 Mon Sep 17 00:00:00 2001 From: Kapil Kumar Singh Date: Thu, 23 Mar 2023 11:06:00 +0530 Subject: [PATCH 6/6] Fix test --- .../sql/catalyst/optimizer/Optimizer.scala | 8 +- .../q14a.sf100/explain.txt | 491 ++++++++------- .../q14a.sf100/simplified.txt | 107 ++-- .../approved-plans-v1_4/q14a/explain.txt | 250 ++++---- .../approved-plans-v1_4/q14a/simplified.txt | 86 +-- .../q14b.sf100/explain.txt | 471 +++++++------- .../q14b.sf100/simplified.txt | 107 ++-- .../approved-plans-v1_4/q14b/explain.txt | 250 ++++---- .../approved-plans-v1_4/q14b/simplified.txt | 86 +-- .../approved-plans-v2_7/q14.sf100/explain.txt | 471 +++++++------- .../q14.sf100/simplified.txt | 107 ++-- .../approved-plans-v2_7/q14/explain.txt | 250 ++++---- .../approved-plans-v2_7/q14/simplified.txt | 86 +-- .../q14a.sf100/explain.txt | 591 +++++++++--------- .../q14a.sf100/simplified.txt | 107 ++-- .../approved-plans-v2_7/q14a/explain.txt | 250 ++++---- .../approved-plans-v2_7/q14a/simplified.txt | 86 +-- .../tpch-plan-stability/q20/explain.txt | 155 ++--- .../tpch-plan-stability/q20/simplified.txt | 51 +- 19 files changed, 2019 insertions(+), 1991 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 6a9e7d5b7935b..32201aa3626b8 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 @@ -172,16 +172,16 @@ abstract class Optimizer(catalogManager: CatalogManager) Batch("Pullup Correlated Expressions", Once, OptimizeOneRowRelationSubquery, PullupCorrelatedPredicates) :: - // Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense - // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). - Batch("Subquery", FixedPoint(1), - OptimizeSubqueries) :: Batch("Preparation for RewriteSubquery", fixedPoint, // Boolean simplification is done before RewritePredicateSubquery so that predicates // containing IN and EXISTS are simplified before rewriting. Eg. NOT(NOT IN) = IN BooleanSimplification) :: Batch("RewriteSubquery", Once, RewritePredicateSubquery) :: + // Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense + // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). + Batch("Subquery", FixedPoint(1), + OptimizeSubqueries) :: Batch("Replace Operators", fixedPoint, RewriteExceptAll, RewriteIntersectAll, 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 d4e832116e90c..4959670c6f6db 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 @@ -1,110 +1,111 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Expand (102) - +- Union (101) - :- * Project (68) - : +- * Filter (67) - : +- * HashAggregate (66) - : +- Exchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (107) ++- * HashAggregate (106) + +- Exchange (105) + +- * HashAggregate (104) + +- * Expand (103) + +- Union (102) + :- * Project (69) + : +- * Filter (68) + : +- * HashAggregate (67) + : +- Exchange (66) + : +- * HashAggregate (65) + : +- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (58) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (57) + : : : +- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * SortMergeJoin LeftSemi (52) - : : : :- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * HashAggregate (38) - : : : : +- Exchange (37) - : : : : +- * HashAggregate (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (33) - : : : : +- * SortMergeJoin LeftSemi (32) - : : : : :- * Sort (17) - : : : : : +- Exchange (16) - : : : : : +- * Filter (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- Scan parquet spark_catalog.default.item (13) - : : : : +- * Sort (31) - : : : : +- Exchange (30) - : : : : +- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Filter (20) - : : : : : : +- * ColumnarToRow (19) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : : : : +- ReusedExchange (21) - : : : : +- BroadcastExchange (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet spark_catalog.default.item (24) - : : : +- * Sort (51) - : : : +- Exchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * Project (46) - : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : :- * Filter (43) - : : : : : +- * ColumnarToRow (42) - : : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- * Project (84) - : +- * Filter (83) - : +- * HashAggregate (82) - : +- Exchange (81) - : +- * HashAggregate (80) - : +- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * Project (76) - : : +- * BroadcastHashJoin Inner BuildRight (75) - : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (74) - : +- ReusedExchange (77) - +- * Project (100) - +- * Filter (99) - +- * HashAggregate (98) - +- Exchange (97) - +- * HashAggregate (96) - +- * Project (95) - +- * BroadcastHashJoin Inner BuildRight (94) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * BroadcastHashJoin LeftSemi BuildRight (89) - : : :- * Filter (87) - : : : +- * ColumnarToRow (86) - : : : +- Scan parquet spark_catalog.default.web_sales (85) - : : +- ReusedExchange (88) - : +- ReusedExchange (90) - +- ReusedExchange (93) + : : : +- BroadcastExchange (54) + : : : +- * SortMergeJoin LeftSemi (53) + : : : :- * Sort (41) + : : : : +- Exchange (40) + : : : : +- * HashAggregate (39) + : : : : +- Exchange (38) + : : : : +- * HashAggregate (37) + : : : : +- * Project (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- ReusedExchange (10) + : : : : +- BroadcastExchange (33) + : : : : +- * SortMergeJoin LeftSemi (32) + : : : : :- * Sort (17) + : : : : : +- Exchange (16) + : : : : : +- * Filter (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- Scan parquet spark_catalog.default.item (13) + : : : : +- * Sort (31) + : : : : +- Exchange (30) + : : : : +- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Filter (20) + : : : : : : +- * ColumnarToRow (19) + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : : : : +- ReusedExchange (21) + : : : : +- BroadcastExchange (27) + : : : : +- * Filter (26) + : : : : +- * ColumnarToRow (25) + : : : : +- Scan parquet spark_catalog.default.item (24) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet spark_catalog.default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + :- * Project (85) + : +- * Filter (84) + : +- * HashAggregate (83) + : +- Exchange (82) + : +- * HashAggregate (81) + : +- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * BroadcastHashJoin LeftSemi BuildRight (74) + : : : :- * Filter (72) + : : : : +- * ColumnarToRow (71) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (70) + : : : +- ReusedExchange (73) + : : +- ReusedExchange (75) + : +- ReusedExchange (78) + +- * Project (101) + +- * Filter (100) + +- * HashAggregate (99) + +- Exchange (98) + +- * HashAggregate (97) + +- * Project (96) + +- * BroadcastHashJoin Inner BuildRight (95) + :- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * BroadcastHashJoin LeftSemi BuildRight (90) + : : :- * Filter (88) + : : : +- * ColumnarToRow (87) + : : : +- Scan parquet spark_catalog.default.web_sales (86) + : : +- ReusedExchange (89) + : +- ReusedExchange (91) + +- ReusedExchange (94) (1) Scan parquet spark_catalog.default.store_sales @@ -151,7 +152,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 135] +(10) ReusedExchange [Reuses operator id: 136] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -201,7 +202,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 135] +(21) ReusedExchange [Reuses operator id: 136] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -267,36 +268,40 @@ Join type: Inner Join condition: None (35) Project [codegen id : 9] -Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Output [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17] -(36) HashAggregate [codegen id : 9] +(36) Project [codegen id : 9] +Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Input [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] + +(37) HashAggregate [codegen id : 9] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(37) Exchange +(38) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(39) Exchange +(40) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 11] +(41) Sort [codegen id : 11] Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.web_sales +(42) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] @@ -304,130 +309,130 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 14] +(43) ColumnarToRow [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(43) Filter [codegen id : 14] +(44) Filter [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 135] +(45) ReusedExchange [Reuses operator id: 136] Output [1]: [d_date_sk#30] -(45) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(46) Project [codegen id : 14] +(47) Project [codegen id : 14] Output [1]: [ws_item_sk#28] Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30] -(47) ReusedExchange [Reuses operator id: 27] +(48) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(48) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 14] +(50) Project [codegen id : 14] Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(50) Exchange +(51) Exchange Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) Sort [codegen id : 15] +(52) Sort [codegen id : 15] Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 16] +(53) SortMergeJoin [codegen id : 16] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)] Join type: LeftSemi Join condition: None -(53) BroadcastExchange +(54) BroadcastExchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8] -(54) BroadcastHashJoin [codegen id : 17] +(55) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(55) Project [codegen id : 17] +(56) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] -(56) BroadcastExchange +(57) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 20] +(58) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 130] +(59) ReusedExchange [Reuses operator id: 131] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 20] +(60) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 20] +(61) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] -(61) ReusedExchange [Reuses operator id: 27] +(62) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(62) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(63) Project [codegen id : 20] +(64) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(64) HashAggregate [codegen id : 20] +(65) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(65) Exchange +(66) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(66) HashAggregate [codegen id : 21] +(67) HashAggregate [codegen id : 21] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50] -(67) Filter [codegen id : 21] +(68) Filter [codegen id : 21] Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50] Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(68) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#38 AS i_brand_id#54, i_class_id#39 AS i_class_id#55, i_category_id#40 AS i_category_id#56] Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50] -(69) Scan parquet spark_catalog.default.catalog_sales +(70) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] @@ -435,75 +440,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 41] +(71) ColumnarToRow [codegen id : 41] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(71) Filter [codegen id : 41] +(72) Filter [codegen id : 41] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Condition : isnotnull(cs_item_sk#57) -(72) ReusedExchange [Reuses operator id: 56] +(73) ReusedExchange [Reuses operator id: 57] Output [1]: [ss_item_sk#35] -(73) BroadcastHashJoin [codegen id : 41] +(74) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(74) ReusedExchange [Reuses operator id: 130] +(75) ReusedExchange [Reuses operator id: 131] Output [1]: [d_date_sk#61] -(75) BroadcastHashJoin [codegen id : 41] +(76) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#61] Join type: Inner Join condition: None -(76) Project [codegen id : 41] +(77) Project [codegen id : 41] Output [3]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59] Input [5]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, d_date_sk#61] -(77) ReusedExchange [Reuses operator id: 27] +(78) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(78) BroadcastHashJoin [codegen id : 41] +(79) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#57] Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(79) Project [codegen id : 41] +(80) Project [codegen id : 41] Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Input [7]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(80) HashAggregate [codegen id : 41] +(81) HashAggregate [codegen id : 41] Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71] -(81) Exchange +(82) Exchange Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71] Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(82) HashAggregate [codegen id : 42] +(83) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73] Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75] -(83) Filter [codegen id : 42] +(84) Filter [codegen id : 42] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(84) Project [codegen id : 42] +(85) Project [codegen id : 42] Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#63, i_class_id#64, i_category_id#65] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75] -(85) Scan parquet spark_catalog.default.web_sales +(86) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] @@ -511,291 +516,291 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 62] +(87) ColumnarToRow [codegen id : 62] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -(87) Filter [codegen id : 62] +(88) Filter [codegen id : 62] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Condition : isnotnull(ws_item_sk#77) -(88) ReusedExchange [Reuses operator id: 56] +(89) ReusedExchange [Reuses operator id: 57] Output [1]: [ss_item_sk#35] -(89) BroadcastHashJoin [codegen id : 62] +(90) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#77] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(90) ReusedExchange [Reuses operator id: 130] +(91) ReusedExchange [Reuses operator id: 131] Output [1]: [d_date_sk#81] -(91) BroadcastHashJoin [codegen id : 62] +(92) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_sold_date_sk#80] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(92) Project [codegen id : 62] +(93) Project [codegen id : 62] Output [3]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79] Input [5]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, d_date_sk#81] -(93) ReusedExchange [Reuses operator id: 27] +(94) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(94) BroadcastHashJoin [codegen id : 62] +(95) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#77] Right keys [1]: [i_item_sk#82] Join type: Inner Join condition: None -(95) Project [codegen id : 62] +(96) Project [codegen id : 62] Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] Input [7]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] -(96) HashAggregate [codegen id : 62] +(97) HashAggregate [codegen id : 62] Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88] Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91] -(97) Exchange +(98) Exchange Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91] Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(98) HashAggregate [codegen id : 63] +(99) HashAggregate [codegen id : 63] Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91] Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93] Results [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95] -(99) Filter [codegen id : 63] +(100) Filter [codegen id : 63] Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95] Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(100) Project [codegen id : 63] +(101) Project [codegen id : 63] Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#83, i_class_id#84, i_category_id#85] Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95] -(101) Union +(102) Union -(102) Expand [codegen id : 64] +(103) Expand [codegen id : 64] Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -(103) HashAggregate [codegen id : 64] +(104) HashAggregate [codegen id : 64] Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -(104) Exchange +(105) Exchange Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(105) HashAggregate [codegen id : 65] +(106) HashAggregate [codegen id : 65] Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [sum(sales#49), sum(number_sales#50)] Aggregate Attributes [2]: [sum(sales#49)#108, sum(number_sales#50)#109] Results [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales#49)#108 AS sum(sales)#110, sum(number_sales#50)#109 AS sum(number_sales)#111] -(106) TakeOrderedAndProject +(107) TakeOrderedAndProject Input [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_class_id#99 ASC NULLS FIRST, i_category_id#100 ASC NULLS FIRST], [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] ===== Subqueries ===== -Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* HashAggregate (125) -+- Exchange (124) - +- * HashAggregate (123) - +- Union (122) - :- * Project (111) - : +- * BroadcastHashJoin Inner BuildRight (110) - : :- * ColumnarToRow (108) - : : +- Scan parquet spark_catalog.default.store_sales (107) - : +- ReusedExchange (109) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * ColumnarToRow (113) - : : +- Scan parquet spark_catalog.default.catalog_sales (112) - : +- ReusedExchange (114) - +- * Project (121) - +- * BroadcastHashJoin Inner BuildRight (120) - :- * ColumnarToRow (118) - : +- Scan parquet spark_catalog.default.web_sales (117) - +- ReusedExchange (119) - - -(107) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* HashAggregate (126) ++- Exchange (125) + +- * HashAggregate (124) + +- Union (123) + :- * Project (112) + : +- * BroadcastHashJoin Inner BuildRight (111) + : :- * ColumnarToRow (109) + : : +- Scan parquet spark_catalog.default.store_sales (108) + : +- ReusedExchange (110) + :- * Project (117) + : +- * BroadcastHashJoin Inner BuildRight (116) + : :- * ColumnarToRow (114) + : : +- Scan parquet spark_catalog.default.catalog_sales (113) + : +- ReusedExchange (115) + +- * Project (122) + +- * BroadcastHashJoin Inner BuildRight (121) + :- * ColumnarToRow (119) + : +- Scan parquet spark_catalog.default.web_sales (118) + +- ReusedExchange (120) + + +(108) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 2] +(109) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] -(109) ReusedExchange [Reuses operator id: 135] +(110) ReusedExchange [Reuses operator id: 136] Output [1]: [d_date_sk#115] -(110) BroadcastHashJoin [codegen id : 2] +(111) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#114] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(111) Project [codegen id : 2] +(112) Project [codegen id : 2] Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117] Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115] -(112) Scan parquet spark_catalog.default.catalog_sales +(113) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)] ReadSchema: struct -(113) ColumnarToRow [codegen id : 4] +(114) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] -(114) ReusedExchange [Reuses operator id: 135] +(115) ReusedExchange [Reuses operator id: 136] Output [1]: [d_date_sk#121] -(115) BroadcastHashJoin [codegen id : 4] +(116) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#120] Right keys [1]: [d_date_sk#121] Join type: Inner Join condition: None -(116) Project [codegen id : 4] +(117) Project [codegen id : 4] Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123] Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121] -(117) Scan parquet spark_catalog.default.web_sales +(118) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 6] +(119) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] -(119) ReusedExchange [Reuses operator id: 135] +(120) ReusedExchange [Reuses operator id: 136] Output [1]: [d_date_sk#127] -(120) BroadcastHashJoin [codegen id : 6] +(121) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#126] Right keys [1]: [d_date_sk#127] Join type: Inner Join condition: None -(121) Project [codegen id : 6] +(122) Project [codegen id : 6] Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129] Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127] -(122) Union +(123) Union -(123) HashAggregate [codegen id : 7] +(124) HashAggregate [codegen id : 7] Input [2]: [quantity#116, list_price#117] Keys: [] Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] Aggregate Attributes [2]: [sum#130, count#131] Results [2]: [sum#132, count#133] -(124) Exchange +(125) Exchange Input [2]: [sum#132, count#133] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(125) HashAggregate [codegen id : 8] +(126) HashAggregate [codegen id : 8] Input [2]: [sum#132, count#133] Keys: [] Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] Aggregate Attributes [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134] Results [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134 AS average_sales#135] -Subquery:2 Hosting operator id = 107 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 108 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 112 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 113 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 117 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 118 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (130) -+- * Project (129) - +- * Filter (128) - +- * ColumnarToRow (127) - +- Scan parquet spark_catalog.default.date_dim (126) +BroadcastExchange (131) ++- * Project (130) + +- * Filter (129) + +- * ColumnarToRow (128) + +- Scan parquet spark_catalog.default.date_dim (127) -(126) Scan parquet spark_catalog.default.date_dim +(127) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#36, d_year#136, d_moy#137] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 1] +(128) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#36, d_year#136, d_moy#137] -(128) Filter [codegen id : 1] +(129) Filter [codegen id : 1] Input [3]: [d_date_sk#36, d_year#136, d_moy#137] Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 = 2001)) AND (d_moy#137 = 11)) AND isnotnull(d_date_sk#36)) -(129) Project [codegen id : 1] +(130) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [3]: [d_date_sk#36, d_year#136, d_moy#137] -(130) BroadcastExchange +(131) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (135) -+- * Project (134) - +- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet spark_catalog.default.date_dim (131) +BroadcastExchange (136) ++- * Project (135) + +- * Filter (134) + +- * ColumnarToRow (133) + +- Scan parquet spark_catalog.default.date_dim (132) -(131) Scan parquet spark_catalog.default.date_dim +(132) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#138] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 1] +(133) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#138] -(133) Filter [codegen id : 1] +(134) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#138] Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#13)) -(134) Project [codegen id : 1] +(135) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#138] -(135) BroadcastExchange +(136) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 99 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 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 cf2a9d9a0b282..14cfabd31b127 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 @@ -94,61 +94,62 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (9) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - 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] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (3) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #10 - WholeStageCodegen (2) - 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] - InputAdapter - WholeStageCodegen (7) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (5) - Filter [i_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + 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] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (3) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (2) + 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] + InputAdapter + WholeStageCodegen (7) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter WholeStageCodegen (15) Sort [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 b187a11721080..ec7e8835546ec 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 @@ -25,46 +25,46 @@ TakeOrderedAndProject (100) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (47) - : : : +- * HashAggregate (46) - : : : +- Exchange (45) - : : : +- * HashAggregate (44) - : : : +- * Project (43) - : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : :- * Project (40) - : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : :- * Filter (9) - : : : : : +- * ColumnarToRow (8) - : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : +- BroadcastExchange (38) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) - : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- BroadcastExchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (41) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Filter (38) + : : : : : +- * ColumnarToRow (37) + : : : : : +- Scan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) : : +- ReusedExchange (52) : +- ReusedExchange (55) :- * Project (78) @@ -138,10 +138,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 9] +(8) ColumnarToRow [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 9] +(9) Filter [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -152,10 +152,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 7] +(11) ColumnarToRow [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 7] +(12) Filter [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -219,121 +219,121 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 7] +(26) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(28) ColumnarToRow [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(29) Filter [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Condition : isnotnull(ws_item_sk#24) +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +(30) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#24] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#24] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] -Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] -(33) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#30] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#25] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None +(34) Exchange +Input [3]: [brand_id#25, class_id#26, category_id#27] +Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) Project [codegen id : 6] -Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(36) BroadcastExchange -Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(37) BroadcastHashJoin [codegen id : 7] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] -Join type: LeftSemi -Join condition: None +(37) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(38) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(38) Filter [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Condition : isnotnull(ws_item_sk#28) -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#28] +Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] +Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#31] +(42) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#34] -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#31] +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(43) Project [codegen id : 9] -Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] - -(44) HashAggregate [codegen id : 9] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] -(45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(45) BroadcastExchange +Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] +Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] +Join type: LeftSemi +Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] (50) BroadcastExchange Input [1]: [ss_item_sk#35] @@ -735,30 +735,30 @@ BroadcastExchange (129) (125) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_year#138] +Output [2]: [d_date_sk#24, d_year#138] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (126) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#138] +Input [2]: [d_date_sk#24, d_year#138] (127) Filter [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#138] -Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#24, d_year#138] +Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#24)) (128) Project [codegen id : 1] -Output [1]: [d_date_sk#31] -Input [2]: [d_date_sk#31, d_year#138] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#138] (129) BroadcastExchange -Input [1]: [d_date_sk#31] +Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 Subquery:9 Hosting operator id = 77 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] 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 9e38c878c274c..0d863814e935e 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 @@ -81,31 +81,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #5 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - 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 #8 - WholeStageCodegen (7) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + 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 #8 + WholeStageCodegen (4) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -132,24 +132,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter 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 f7de3cf995f55..5d8dba698a65a 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 @@ -1,92 +1,93 @@ == Physical Plan == -TakeOrderedAndProject (88) -+- * BroadcastHashJoin Inner BuildRight (87) - :- * Filter (70) - : +- * HashAggregate (69) - : +- Exchange (68) - : +- * HashAggregate (67) - : +- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (89) ++- * BroadcastHashJoin Inner BuildRight (88) + :- * Filter (71) + : +- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (58) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (57) + : : : +- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * SortMergeJoin LeftSemi (52) - : : : :- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * HashAggregate (38) - : : : : +- Exchange (37) - : : : : +- * HashAggregate (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (33) - : : : : +- * SortMergeJoin LeftSemi (32) - : : : : :- * Sort (17) - : : : : : +- Exchange (16) - : : : : : +- * Filter (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- Scan parquet spark_catalog.default.item (13) - : : : : +- * Sort (31) - : : : : +- Exchange (30) - : : : : +- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Filter (20) - : : : : : : +- * ColumnarToRow (19) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : : : : +- ReusedExchange (21) - : : : : +- BroadcastExchange (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet spark_catalog.default.item (24) - : : : +- * Sort (51) - : : : +- Exchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * Project (46) - : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : :- * Filter (43) - : : : : : +- * ColumnarToRow (42) - : : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (58) - : +- BroadcastExchange (64) - : +- * Filter (63) - : +- * ColumnarToRow (62) - : +- Scan parquet spark_catalog.default.item (61) - +- BroadcastExchange (86) - +- * Filter (85) - +- * HashAggregate (84) - +- Exchange (83) - +- * HashAggregate (82) - +- * Project (81) - +- * BroadcastHashJoin Inner BuildRight (80) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin LeftSemi BuildRight (75) - : : :- * Filter (73) - : : : +- * ColumnarToRow (72) - : : : +- Scan parquet spark_catalog.default.store_sales (71) - : : +- ReusedExchange (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) + : : : +- BroadcastExchange (54) + : : : +- * SortMergeJoin LeftSemi (53) + : : : :- * Sort (41) + : : : : +- Exchange (40) + : : : : +- * HashAggregate (39) + : : : : +- Exchange (38) + : : : : +- * HashAggregate (37) + : : : : +- * Project (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- ReusedExchange (10) + : : : : +- BroadcastExchange (33) + : : : : +- * SortMergeJoin LeftSemi (32) + : : : : :- * Sort (17) + : : : : : +- Exchange (16) + : : : : : +- * Filter (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- Scan parquet spark_catalog.default.item (13) + : : : : +- * Sort (31) + : : : : +- Exchange (30) + : : : : +- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Filter (20) + : : : : : : +- * ColumnarToRow (19) + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : : : : +- ReusedExchange (21) + : : : : +- BroadcastExchange (27) + : : : : +- * Filter (26) + : : : : +- * ColumnarToRow (25) + : : : : +- Scan parquet spark_catalog.default.item (24) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet spark_catalog.default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (59) + : +- BroadcastExchange (65) + : +- * Filter (64) + : +- * ColumnarToRow (63) + : +- Scan parquet spark_catalog.default.item (62) + +- BroadcastExchange (87) + +- * Filter (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * BroadcastHashJoin LeftSemi BuildRight (76) + : : :- * Filter (74) + : : : +- * ColumnarToRow (73) + : : : +- Scan parquet spark_catalog.default.store_sales (72) + : : +- ReusedExchange (75) + : +- ReusedExchange (77) + +- ReusedExchange (80) (1) Scan parquet spark_catalog.default.store_sales @@ -133,7 +134,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 121] +(10) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -183,7 +184,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 121] +(21) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -249,36 +250,40 @@ Join type: Inner Join condition: None (35) Project [codegen id : 9] -Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Output [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17] -(36) HashAggregate [codegen id : 9] +(36) Project [codegen id : 9] +Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Input [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] + +(37) HashAggregate [codegen id : 9] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(37) Exchange +(38) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(39) Exchange +(40) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 11] +(41) Sort [codegen id : 11] Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.web_sales +(42) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] @@ -286,141 +291,141 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 14] +(43) ColumnarToRow [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(43) Filter [codegen id : 14] +(44) Filter [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 121] +(45) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#30] -(45) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(46) Project [codegen id : 14] +(47) Project [codegen id : 14] Output [1]: [ws_item_sk#28] Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30] -(47) ReusedExchange [Reuses operator id: 27] +(48) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(48) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 14] +(50) Project [codegen id : 14] Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(50) Exchange +(51) Exchange Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) Sort [codegen id : 15] +(52) Sort [codegen id : 15] Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 16] +(53) SortMergeJoin [codegen id : 16] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)] Join type: LeftSemi Join condition: None -(53) BroadcastExchange +(54) BroadcastExchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8] -(54) BroadcastHashJoin [codegen id : 17] +(55) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(55) Project [codegen id : 17] +(56) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] -(56) BroadcastExchange +(57) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 20] +(58) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 112] +(59) ReusedExchange [Reuses operator id: 113] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 20] +(60) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 20] +(61) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] -(61) Scan parquet spark_catalog.default.item +(62) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 19] +(63) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(63) Filter [codegen id : 19] +(64) Filter [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40)) -(64) BroadcastExchange +(65) BroadcastExchange Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(65) BroadcastHashJoin [codegen id : 20] +(66) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(66) Project [codegen id : 20] +(67) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(67) HashAggregate [codegen id : 20] +(68) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(68) Exchange +(69) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(69) HashAggregate [codegen id : 42] +(70) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(70) Filter [codegen id : 42] +(71) Filter [codegen id : 42] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(71) Scan parquet spark_catalog.default.store_sales +(72) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -428,346 +433,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 40] +(73) ColumnarToRow [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(73) Filter [codegen id : 40] +(74) Filter [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) -(74) ReusedExchange [Reuses operator id: 56] +(75) ReusedExchange [Reuses operator id: 57] Output [1]: [ss_item_sk#35] -(75) BroadcastHashJoin [codegen id : 40] +(76) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(76) ReusedExchange [Reuses operator id: 126] +(77) ReusedExchange [Reuses operator id: 127] Output [1]: [d_date_sk#59] -(77) BroadcastHashJoin [codegen id : 40] +(78) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(78) Project [codegen id : 40] +(79) Project [codegen id : 40] Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56] Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59] -(79) ReusedExchange [Reuses operator id: 64] +(80) ReusedExchange [Reuses operator id: 65] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(80) BroadcastHashJoin [codegen id : 40] +(81) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(81) Project [codegen id : 40] +(82) Project [codegen id : 40] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(82) HashAggregate [codegen id : 40] +(83) HashAggregate [codegen id : 40] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] -(83) Exchange +(84) Exchange Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(84) HashAggregate [codegen id : 41] +(85) HashAggregate [codegen id : 41] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(85) Filter [codegen id : 41] +(86) Filter [codegen id : 41] Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(86) BroadcastExchange +(87) BroadcastExchange Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13] -(87) BroadcastHashJoin [codegen id : 42] +(88) BroadcastHashJoin [codegen id : 42] Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner Join condition: None -(88) TakeOrderedAndProject +(89) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: 100, [i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- Exchange (106) - +- * HashAggregate (105) - +- Union (104) - :- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * ColumnarToRow (90) - : : +- Scan parquet spark_catalog.default.store_sales (89) - : +- ReusedExchange (91) - :- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * ColumnarToRow (95) - : : +- Scan parquet spark_catalog.default.catalog_sales (94) - : +- ReusedExchange (96) - +- * Project (103) - +- * BroadcastHashJoin Inner BuildRight (102) - :- * ColumnarToRow (100) - : +- Scan parquet spark_catalog.default.web_sales (99) - +- ReusedExchange (101) - - -(89) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 71 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (108) ++- Exchange (107) + +- * HashAggregate (106) + +- Union (105) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- Scan parquet spark_catalog.default.store_sales (90) + : +- ReusedExchange (92) + :- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * ColumnarToRow (96) + : : +- Scan parquet spark_catalog.default.catalog_sales (95) + : +- ReusedExchange (97) + +- * Project (104) + +- * BroadcastHashJoin Inner BuildRight (103) + :- * ColumnarToRow (101) + : +- Scan parquet spark_catalog.default.web_sales (100) + +- ReusedExchange (102) + + +(90) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(90) ColumnarToRow [codegen id : 2] +(91) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(91) ReusedExchange [Reuses operator id: 121] +(92) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#78] -(92) BroadcastHashJoin [codegen id : 2] +(93) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(93) Project [codegen id : 2] +(94) Project [codegen id : 2] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] -(94) Scan parquet spark_catalog.default.catalog_sales +(95) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 4] +(96) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -(96) ReusedExchange [Reuses operator id: 121] +(97) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#84] -(97) BroadcastHashJoin [codegen id : 4] +(98) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(98) Project [codegen id : 4] +(99) Project [codegen id : 4] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] -(99) Scan parquet spark_catalog.default.web_sales +(100) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 6] +(101) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(101) ReusedExchange [Reuses operator id: 121] +(102) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#90] -(102) BroadcastHashJoin [codegen id : 6] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(103) Project [codegen id : 6] +(104) Project [codegen id : 6] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] -(104) Union +(105) Union -(105) HashAggregate [codegen id : 7] +(106) HashAggregate [codegen id : 7] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [2]: [sum#93, count#94] Results [2]: [sum#95, count#96] -(106) Exchange +(107) Exchange Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(107) HashAggregate [codegen id : 8] +(108) HashAggregate [codegen id : 8] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] -Subquery:2 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 94 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * Project (111) - +- * Filter (110) - +- * ColumnarToRow (109) - +- Scan parquet spark_catalog.default.date_dim (108) +BroadcastExchange (113) ++- * Project (112) + +- * Filter (111) + +- * ColumnarToRow (110) + +- Scan parquet spark_catalog.default.date_dim (109) -(108) Scan parquet spark_catalog.default.date_dim +(109) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#36, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(109) ColumnarToRow [codegen id : 1] +(110) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] -(110) Filter [codegen id : 1] +(111) Filter [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36)) -(111) Project [codegen id : 1] +(112) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [2]: [d_date_sk#36, d_week_seq#99] -(112) BroadcastExchange +(113) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:6 Hosting operator id = 110 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* Project (116) -+- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet spark_catalog.default.date_dim (113) +Subquery:6 Hosting operator id = 111 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* Project (117) ++- * Filter (116) + +- * ColumnarToRow (115) + +- Scan parquet spark_catalog.default.date_dim (114) -(113) Scan parquet spark_catalog.default.date_dim +(114) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -(115) Filter [codegen id : 1] +(116) Filter [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) -(116) Project [codegen id : 1] +(117) Project [codegen id : 1] Output [1]: [d_week_seq#102] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * Project (120) - +- * Filter (119) - +- * ColumnarToRow (118) - +- Scan parquet spark_catalog.default.date_dim (117) +BroadcastExchange (122) ++- * Project (121) + +- * Filter (120) + +- * ColumnarToRow (119) + +- Scan parquet spark_catalog.default.date_dim (118) -(117) Scan parquet spark_catalog.default.date_dim +(118) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 1] +(119) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] -(119) Filter [codegen id : 1] +(120) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#13)) -(120) Project [codegen id : 1] +(121) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#106] -(121) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * Project (125) - +- * Filter (124) - +- * ColumnarToRow (123) - +- Scan parquet spark_catalog.default.date_dim (122) +Subquery:11 Hosting operator id = 72 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (127) ++- * Project (126) + +- * Filter (125) + +- * ColumnarToRow (124) + +- Scan parquet spark_catalog.default.date_dim (123) -(122) Scan parquet spark_catalog.default.date_dim +(123) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#59, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(123) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] -(124) Filter [codegen id : 1] +(125) Filter [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59)) -(125) Project [codegen id : 1] +(126) Project [codegen id : 1] Output [1]: [d_date_sk#59] Input [2]: [d_date_sk#59, d_week_seq#107] -(126) BroadcastExchange +(127) BroadcastExchange Input [1]: [d_date_sk#59] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -Subquery:12 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#108, [id=#109] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet spark_catalog.default.date_dim (127) +Subquery:12 Hosting operator id = 125 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* Project (131) ++- * Filter (130) + +- * ColumnarToRow (129) + +- Scan parquet spark_catalog.default.date_dim (128) -(127) Scan parquet spark_catalog.default.date_dim +(128) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 1] +(129) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -(129) Filter [codegen id : 1] +(130) Filter [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11)) -(130) Project [codegen id : 1] +(131) Project [codegen id : 1] Output [1]: [d_week_seq#110] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] 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 005d0cc8cfb42..b13aaa6c45bd2 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 @@ -92,61 +92,62 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (9) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #7 - 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] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (3) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (2) - 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] - InputAdapter - WholeStageCodegen (7) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #10 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (5) - Filter [i_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #7 + 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] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (3) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #9 + WholeStageCodegen (2) + 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] + InputAdapter + WholeStageCodegen (7) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter WholeStageCodegen (15) Sort [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 d95e18602d24d..3d19a8ba1d58c 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 @@ -20,46 +20,46 @@ TakeOrderedAndProject (82) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (47) - : : : +- * HashAggregate (46) - : : : +- Exchange (45) - : : : +- * HashAggregate (44) - : : : +- * Project (43) - : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : :- * Project (40) - : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : :- * Filter (9) - : : : : : +- * ColumnarToRow (8) - : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : +- BroadcastExchange (38) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) - : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- BroadcastExchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (41) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Filter (38) + : : : : : +- * ColumnarToRow (37) + : : : : : +- Scan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) : : +- BroadcastExchange (55) : : +- * Filter (54) : : +- * ColumnarToRow (53) @@ -120,10 +120,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 9] +(8) ColumnarToRow [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 9] +(9) Filter [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -134,10 +134,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 7] +(11) ColumnarToRow [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 7] +(12) Filter [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -201,121 +201,121 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 7] +(26) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(28) ColumnarToRow [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(29) Filter [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Condition : isnotnull(ws_item_sk#24) +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +(30) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#24] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#24] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] -Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] -(33) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#30] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#25] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None +(34) Exchange +Input [3]: [brand_id#25, class_id#26, category_id#27] +Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) Project [codegen id : 6] -Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(36) BroadcastExchange -Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(37) BroadcastHashJoin [codegen id : 7] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] -Join type: LeftSemi -Join condition: None +(37) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(38) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(38) Filter [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Condition : isnotnull(ws_item_sk#28) -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#28] +Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] +Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#31] +(42) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#34] -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#31] +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(43) Project [codegen id : 9] -Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] - -(44) HashAggregate [codegen id : 9] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] -(45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(45) BroadcastExchange +Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] +Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] +Join type: LeftSemi +Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] (50) BroadcastExchange Input [1]: [ss_item_sk#35] @@ -659,30 +659,30 @@ BroadcastExchange (115) (111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_year#106] +Output [2]: [d_date_sk#24, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (112) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#106] +Input [2]: [d_date_sk#24, d_year#106] (113) Filter [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#106] -Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#24, d_year#106] +Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#24)) (114) Project [codegen id : 1] -Output [1]: [d_date_sk#31] -Input [2]: [d_date_sk#31, d_year#106] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#106] (115) BroadcastExchange -Input [1]: [d_date_sk#31] +Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 Subquery:10 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] 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 825bb2d20a585..0e5a5148dd045 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 @@ -79,31 +79,31 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - 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 #7 - WholeStageCodegen (7) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + 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 #7 + WholeStageCodegen (4) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -130,24 +130,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 WholeStageCodegen (12) 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 71b0eb2242e54..d3598f92d05cd 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 @@ -1,92 +1,93 @@ == Physical Plan == -TakeOrderedAndProject (88) -+- * BroadcastHashJoin Inner BuildRight (87) - :- * Filter (70) - : +- * HashAggregate (69) - : +- Exchange (68) - : +- * HashAggregate (67) - : +- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (89) ++- * BroadcastHashJoin Inner BuildRight (88) + :- * Filter (71) + : +- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * BroadcastHashJoin LeftSemi BuildRight (58) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (57) + : : : +- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * SortMergeJoin LeftSemi (52) - : : : :- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * HashAggregate (38) - : : : : +- Exchange (37) - : : : : +- * HashAggregate (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (33) - : : : : +- * SortMergeJoin LeftSemi (32) - : : : : :- * Sort (17) - : : : : : +- Exchange (16) - : : : : : +- * Filter (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- Scan parquet spark_catalog.default.item (13) - : : : : +- * Sort (31) - : : : : +- Exchange (30) - : : : : +- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Filter (20) - : : : : : : +- * ColumnarToRow (19) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : : : : +- ReusedExchange (21) - : : : : +- BroadcastExchange (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet spark_catalog.default.item (24) - : : : +- * Sort (51) - : : : +- Exchange (50) - : : : +- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * Project (46) - : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : :- * Filter (43) - : : : : : +- * ColumnarToRow (42) - : : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (58) - : +- BroadcastExchange (64) - : +- * Filter (63) - : +- * ColumnarToRow (62) - : +- Scan parquet spark_catalog.default.item (61) - +- BroadcastExchange (86) - +- * Filter (85) - +- * HashAggregate (84) - +- Exchange (83) - +- * HashAggregate (82) - +- * Project (81) - +- * BroadcastHashJoin Inner BuildRight (80) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin LeftSemi BuildRight (75) - : : :- * Filter (73) - : : : +- * ColumnarToRow (72) - : : : +- Scan parquet spark_catalog.default.store_sales (71) - : : +- ReusedExchange (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) + : : : +- BroadcastExchange (54) + : : : +- * SortMergeJoin LeftSemi (53) + : : : :- * Sort (41) + : : : : +- Exchange (40) + : : : : +- * HashAggregate (39) + : : : : +- Exchange (38) + : : : : +- * HashAggregate (37) + : : : : +- * Project (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- ReusedExchange (10) + : : : : +- BroadcastExchange (33) + : : : : +- * SortMergeJoin LeftSemi (32) + : : : : :- * Sort (17) + : : : : : +- Exchange (16) + : : : : : +- * Filter (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- Scan parquet spark_catalog.default.item (13) + : : : : +- * Sort (31) + : : : : +- Exchange (30) + : : : : +- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Filter (20) + : : : : : : +- * ColumnarToRow (19) + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : : : : +- ReusedExchange (21) + : : : : +- BroadcastExchange (27) + : : : : +- * Filter (26) + : : : : +- * ColumnarToRow (25) + : : : : +- Scan parquet spark_catalog.default.item (24) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet spark_catalog.default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (59) + : +- BroadcastExchange (65) + : +- * Filter (64) + : +- * ColumnarToRow (63) + : +- Scan parquet spark_catalog.default.item (62) + +- BroadcastExchange (87) + +- * Filter (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * BroadcastHashJoin LeftSemi BuildRight (76) + : : :- * Filter (74) + : : : +- * ColumnarToRow (73) + : : : +- Scan parquet spark_catalog.default.store_sales (72) + : : +- ReusedExchange (75) + : +- ReusedExchange (77) + +- ReusedExchange (80) (1) Scan parquet spark_catalog.default.store_sales @@ -133,7 +134,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 121] +(10) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -183,7 +184,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 121] +(21) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -249,36 +250,40 @@ Join type: Inner Join condition: None (35) Project [codegen id : 9] -Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Output [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17] -(36) HashAggregate [codegen id : 9] +(36) Project [codegen id : 9] +Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Input [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] + +(37) HashAggregate [codegen id : 9] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(37) Exchange +(38) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(39) Exchange +(40) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 11] +(41) Sort [codegen id : 11] Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.web_sales +(42) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] @@ -286,141 +291,141 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 14] +(43) ColumnarToRow [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(43) Filter [codegen id : 14] +(44) Filter [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 121] +(45) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#30] -(45) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(46) Project [codegen id : 14] +(47) Project [codegen id : 14] Output [1]: [ws_item_sk#28] Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30] -(47) ReusedExchange [Reuses operator id: 27] +(48) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(48) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 14] +(50) Project [codegen id : 14] Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(50) Exchange +(51) Exchange Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) Sort [codegen id : 15] +(52) Sort [codegen id : 15] Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 16] +(53) SortMergeJoin [codegen id : 16] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)] Join type: LeftSemi Join condition: None -(53) BroadcastExchange +(54) BroadcastExchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8] -(54) BroadcastHashJoin [codegen id : 17] +(55) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(55) Project [codegen id : 17] +(56) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] -(56) BroadcastExchange +(57) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 20] +(58) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 112] +(59) ReusedExchange [Reuses operator id: 113] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 20] +(60) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 20] +(61) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] -(61) Scan parquet spark_catalog.default.item +(62) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 19] +(63) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(63) Filter [codegen id : 19] +(64) Filter [codegen id : 19] Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40)) -(64) BroadcastExchange +(65) BroadcastExchange Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(65) BroadcastHashJoin [codegen id : 20] +(66) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(66) Project [codegen id : 20] +(67) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(67) HashAggregate [codegen id : 20] +(68) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(68) Exchange +(69) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(69) HashAggregate [codegen id : 42] +(70) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(70) Filter [codegen id : 42] +(71) Filter [codegen id : 42] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(71) Scan parquet spark_catalog.default.store_sales +(72) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -428,346 +433,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 40] +(73) ColumnarToRow [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(73) Filter [codegen id : 40] +(74) Filter [codegen id : 40] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) -(74) ReusedExchange [Reuses operator id: 56] +(75) ReusedExchange [Reuses operator id: 57] Output [1]: [ss_item_sk#35] -(75) BroadcastHashJoin [codegen id : 40] +(76) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(76) ReusedExchange [Reuses operator id: 126] +(77) ReusedExchange [Reuses operator id: 127] Output [1]: [d_date_sk#59] -(77) BroadcastHashJoin [codegen id : 40] +(78) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(78) Project [codegen id : 40] +(79) Project [codegen id : 40] Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56] Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59] -(79) ReusedExchange [Reuses operator id: 64] +(80) ReusedExchange [Reuses operator id: 65] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(80) BroadcastHashJoin [codegen id : 40] +(81) BroadcastHashJoin [codegen id : 40] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(81) Project [codegen id : 40] +(82) Project [codegen id : 40] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(82) HashAggregate [codegen id : 40] +(83) HashAggregate [codegen id : 40] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] -(83) Exchange +(84) Exchange Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(84) HashAggregate [codegen id : 41] +(85) HashAggregate [codegen id : 41] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(85) Filter [codegen id : 41] +(86) Filter [codegen id : 41] Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(86) BroadcastExchange +(87) BroadcastExchange Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13] -(87) BroadcastHashJoin [codegen id : 42] +(88) BroadcastHashJoin [codegen id : 42] Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner Join condition: None -(88) TakeOrderedAndProject +(89) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] Arguments: 100, [i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- Exchange (106) - +- * HashAggregate (105) - +- Union (104) - :- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * ColumnarToRow (90) - : : +- Scan parquet spark_catalog.default.store_sales (89) - : +- ReusedExchange (91) - :- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * ColumnarToRow (95) - : : +- Scan parquet spark_catalog.default.catalog_sales (94) - : +- ReusedExchange (96) - +- * Project (103) - +- * BroadcastHashJoin Inner BuildRight (102) - :- * ColumnarToRow (100) - : +- Scan parquet spark_catalog.default.web_sales (99) - +- ReusedExchange (101) - - -(89) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 71 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (108) ++- Exchange (107) + +- * HashAggregate (106) + +- Union (105) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- Scan parquet spark_catalog.default.store_sales (90) + : +- ReusedExchange (92) + :- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * ColumnarToRow (96) + : : +- Scan parquet spark_catalog.default.catalog_sales (95) + : +- ReusedExchange (97) + +- * Project (104) + +- * BroadcastHashJoin Inner BuildRight (103) + :- * ColumnarToRow (101) + : +- Scan parquet spark_catalog.default.web_sales (100) + +- ReusedExchange (102) + + +(90) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(90) ColumnarToRow [codegen id : 2] +(91) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(91) ReusedExchange [Reuses operator id: 121] +(92) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#78] -(92) BroadcastHashJoin [codegen id : 2] +(93) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(93) Project [codegen id : 2] +(94) Project [codegen id : 2] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] -(94) Scan parquet spark_catalog.default.catalog_sales +(95) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 4] +(96) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -(96) ReusedExchange [Reuses operator id: 121] +(97) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#84] -(97) BroadcastHashJoin [codegen id : 4] +(98) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(98) Project [codegen id : 4] +(99) Project [codegen id : 4] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] -(99) Scan parquet spark_catalog.default.web_sales +(100) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 6] +(101) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -(101) ReusedExchange [Reuses operator id: 121] +(102) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#90] -(102) BroadcastHashJoin [codegen id : 6] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(103) Project [codegen id : 6] +(104) Project [codegen id : 6] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] -(104) Union +(105) Union -(105) HashAggregate [codegen id : 7] +(106) HashAggregate [codegen id : 7] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [2]: [sum#93, count#94] Results [2]: [sum#95, count#96] -(106) Exchange +(107) Exchange Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(107) HashAggregate [codegen id : 8] +(108) HashAggregate [codegen id : 8] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] -Subquery:2 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 94 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * Project (111) - +- * Filter (110) - +- * ColumnarToRow (109) - +- Scan parquet spark_catalog.default.date_dim (108) +BroadcastExchange (113) ++- * Project (112) + +- * Filter (111) + +- * ColumnarToRow (110) + +- Scan parquet spark_catalog.default.date_dim (109) -(108) Scan parquet spark_catalog.default.date_dim +(109) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#36, d_week_seq#99] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(109) ColumnarToRow [codegen id : 1] +(110) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] -(110) Filter [codegen id : 1] +(111) Filter [codegen id : 1] Input [2]: [d_date_sk#36, d_week_seq#99] Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36)) -(111) Project [codegen id : 1] +(112) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [2]: [d_date_sk#36, d_week_seq#99] -(112) BroadcastExchange +(113) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:6 Hosting operator id = 110 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* Project (116) -+- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet spark_catalog.default.date_dim (113) +Subquery:6 Hosting operator id = 111 Hosting Expression = Subquery scalar-subquery#100, [id=#101] +* Project (117) ++- * Filter (116) + +- * ColumnarToRow (115) + +- Scan parquet spark_catalog.default.date_dim (114) -(113) Scan parquet spark_catalog.default.date_dim +(114) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 1] +(115) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -(115) Filter [codegen id : 1] +(116) Filter [codegen id : 1] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) -(116) Project [codegen id : 1] +(117) Project [codegen id : 1] Output [1]: [d_week_seq#102] Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * Project (120) - +- * Filter (119) - +- * ColumnarToRow (118) - +- Scan parquet spark_catalog.default.date_dim (117) +BroadcastExchange (122) ++- * Project (121) + +- * Filter (120) + +- * ColumnarToRow (119) + +- Scan parquet spark_catalog.default.date_dim (118) -(117) Scan parquet spark_catalog.default.date_dim +(118) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 1] +(119) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] -(119) Filter [codegen id : 1] +(120) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#106] Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#13)) -(120) Project [codegen id : 1] +(121) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#106] -(121) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:10 Hosting operator id = 86 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:11 Hosting operator id = 71 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * Project (125) - +- * Filter (124) - +- * ColumnarToRow (123) - +- Scan parquet spark_catalog.default.date_dim (122) +Subquery:11 Hosting operator id = 72 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 +BroadcastExchange (127) ++- * Project (126) + +- * Filter (125) + +- * ColumnarToRow (124) + +- Scan parquet spark_catalog.default.date_dim (123) -(122) Scan parquet spark_catalog.default.date_dim +(123) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#59, d_week_seq#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(123) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] -(124) Filter [codegen id : 1] +(125) Filter [codegen id : 1] Input [2]: [d_date_sk#59, d_week_seq#107] Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59)) -(125) Project [codegen id : 1] +(126) Project [codegen id : 1] Output [1]: [d_date_sk#59] Input [2]: [d_date_sk#59, d_week_seq#107] -(126) BroadcastExchange +(127) BroadcastExchange Input [1]: [d_date_sk#59] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] -Subquery:12 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#108, [id=#109] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet spark_catalog.default.date_dim (127) +Subquery:12 Hosting operator id = 125 Hosting Expression = Subquery scalar-subquery#108, [id=#109] +* Project (131) ++- * Filter (130) + +- * ColumnarToRow (129) + +- Scan parquet spark_catalog.default.date_dim (128) -(127) Scan parquet spark_catalog.default.date_dim +(128) Scan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 1] +(129) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] -(129) Filter [codegen id : 1] +(130) Filter [codegen id : 1] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16)) -(130) Project [codegen id : 1] +(131) Project [codegen id : 1] Output [1]: [d_week_seq#110] Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113] 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 005d0cc8cfb42..b13aaa6c45bd2 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 @@ -92,61 +92,62 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (9) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #7 - 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] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (3) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (2) - 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] - InputAdapter - WholeStageCodegen (7) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #10 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (5) - Filter [i_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #7 + 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] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (3) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #9 + WholeStageCodegen (2) + 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] + InputAdapter + WholeStageCodegen (7) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter WholeStageCodegen (15) Sort [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 6f28a5b6f2723..3346412db7378 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 @@ -20,46 +20,46 @@ TakeOrderedAndProject (82) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (47) - : : : +- * HashAggregate (46) - : : : +- Exchange (45) - : : : +- * HashAggregate (44) - : : : +- * Project (43) - : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : :- * Project (40) - : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : :- * Filter (9) - : : : : : +- * ColumnarToRow (8) - : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : +- BroadcastExchange (38) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) - : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- BroadcastExchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (41) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Filter (38) + : : : : : +- * ColumnarToRow (37) + : : : : : +- Scan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) : : +- BroadcastExchange (55) : : +- * Filter (54) : : +- * ColumnarToRow (53) @@ -120,10 +120,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 9] +(8) ColumnarToRow [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 9] +(9) Filter [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -134,10 +134,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 7] +(11) ColumnarToRow [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 7] +(12) Filter [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -201,121 +201,121 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 7] +(26) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(28) ColumnarToRow [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(29) Filter [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Condition : isnotnull(ws_item_sk#24) +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +(30) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#24] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#24] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] -Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] -(33) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#30] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#25] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None +(34) Exchange +Input [3]: [brand_id#25, class_id#26, category_id#27] +Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) Project [codegen id : 6] -Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(36) BroadcastExchange -Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(37) BroadcastHashJoin [codegen id : 7] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] -Join type: LeftSemi -Join condition: None +(37) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(38) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(38) Filter [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Condition : isnotnull(ws_item_sk#28) -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#28] +Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] +Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#31] +(42) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#34] -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#31] +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(43) Project [codegen id : 9] -Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] - -(44) HashAggregate [codegen id : 9] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] -(45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(45) BroadcastExchange +Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] +Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] +Join type: LeftSemi +Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] (50) BroadcastExchange Input [1]: [ss_item_sk#35] @@ -659,30 +659,30 @@ BroadcastExchange (115) (111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_year#106] +Output [2]: [d_date_sk#24, d_year#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (112) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#106] +Input [2]: [d_date_sk#24, d_year#106] (113) Filter [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#106] -Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#24, d_year#106] +Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#24)) (114) Project [codegen id : 1] -Output [1]: [d_date_sk#31] -Input [2]: [d_date_sk#31, d_year#106] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#106] (115) BroadcastExchange -Input [1]: [d_date_sk#31] +Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 Subquery:10 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] 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 825bb2d20a585..0e5a5148dd045 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 @@ -79,31 +79,31 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - 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 #7 - WholeStageCodegen (7) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + 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 #7 + WholeStageCodegen (4) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -130,24 +130,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 WholeStageCodegen (12) 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 0358a4084c20a..2ec4741828279 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 @@ -1,130 +1,131 @@ == Physical Plan == -TakeOrderedAndProject (126) -+- * HashAggregate (125) - +- Exchange (124) - +- * HashAggregate (123) - +- Union (122) - :- * HashAggregate (101) - : +- Exchange (100) - : +- * HashAggregate (99) - : +- Union (98) - : :- * Filter (67) - : : +- * HashAggregate (66) - : : +- Exchange (65) - : : +- * HashAggregate (64) - : : +- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (127) ++- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- Union (123) + :- * HashAggregate (102) + : +- Exchange (101) + : +- * HashAggregate (100) + : +- Union (99) + : :- * Filter (68) + : : +- * HashAggregate (67) + : : +- Exchange (66) + : : +- * HashAggregate (65) + : : +- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (58) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : +- BroadcastExchange (57) + : : : : +- * Project (56) + : : : : +- * BroadcastHashJoin Inner BuildRight (55) : : : : :- * Filter (6) : : : : : +- * ColumnarToRow (5) : : : : : +- Scan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (53) - : : : : +- * SortMergeJoin LeftSemi (52) - : : : : :- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * HashAggregate (38) - : : : : : +- Exchange (37) - : : : : : +- * HashAggregate (36) - : : : : : +- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (12) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- ReusedExchange (10) - : : : : : +- BroadcastExchange (33) - : : : : : +- * SortMergeJoin LeftSemi (32) - : : : : : :- * Sort (17) - : : : : : : +- Exchange (16) - : : : : : : +- * Filter (15) - : : : : : : +- * ColumnarToRow (14) - : : : : : : +- Scan parquet spark_catalog.default.item (13) - : : : : : +- * Sort (31) - : : : : : +- Exchange (30) - : : : : : +- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Project (23) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : : :- * Filter (20) - : : : : : : : +- * ColumnarToRow (19) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : : : : : +- ReusedExchange (21) - : : : : : +- BroadcastExchange (27) - : : : : : +- * Filter (26) - : : : : : +- * ColumnarToRow (25) - : : : : : +- Scan parquet spark_catalog.default.item (24) - : : : : +- * Sort (51) - : : : : +- Exchange (50) - : : : : +- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * Project (46) - : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : :- * Filter (43) - : : : : : : +- * ColumnarToRow (42) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : : : +- ReusedExchange (44) - : : : : +- ReusedExchange (47) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : :- * Filter (82) - : : +- * HashAggregate (81) - : : +- Exchange (80) - : : +- * HashAggregate (79) - : : +- * Project (78) - : : +- * BroadcastHashJoin Inner BuildRight (77) - : : :- * Project (75) - : : : +- * BroadcastHashJoin Inner BuildRight (74) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (72) - : : : : :- * Filter (70) - : : : : : +- * ColumnarToRow (69) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (68) - : : : : +- ReusedExchange (71) - : : : +- ReusedExchange (73) - : : +- ReusedExchange (76) - : +- * Filter (97) - : +- * HashAggregate (96) - : +- Exchange (95) - : +- * HashAggregate (94) - : +- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * Project (90) - : : +- * BroadcastHashJoin Inner BuildRight (89) - : : :- * BroadcastHashJoin LeftSemi BuildRight (87) - : : : :- * Filter (85) - : : : : +- * ColumnarToRow (84) - : : : : +- Scan parquet spark_catalog.default.web_sales (83) - : : : +- ReusedExchange (86) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- * HashAggregate (106) - : +- Exchange (105) - : +- * HashAggregate (104) - : +- * HashAggregate (103) - : +- ReusedExchange (102) - :- * HashAggregate (111) - : +- Exchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- ReusedExchange (107) - :- * HashAggregate (116) - : +- Exchange (115) - : +- * HashAggregate (114) - : +- * HashAggregate (113) - : +- ReusedExchange (112) - +- * HashAggregate (121) - +- Exchange (120) - +- * HashAggregate (119) - +- * HashAggregate (118) - +- ReusedExchange (117) + : : : : +- BroadcastExchange (54) + : : : : +- * SortMergeJoin LeftSemi (53) + : : : : :- * Sort (41) + : : : : : +- Exchange (40) + : : : : : +- * HashAggregate (39) + : : : : : +- Exchange (38) + : : : : : +- * HashAggregate (37) + : : : : : +- * Project (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (12) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : : +- ReusedExchange (10) + : : : : : +- BroadcastExchange (33) + : : : : : +- * SortMergeJoin LeftSemi (32) + : : : : : :- * Sort (17) + : : : : : : +- Exchange (16) + : : : : : : +- * Filter (15) + : : : : : : +- * ColumnarToRow (14) + : : : : : : +- Scan parquet spark_catalog.default.item (13) + : : : : : +- * Sort (31) + : : : : : +- Exchange (30) + : : : : : +- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Project (23) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : : :- * Filter (20) + : : : : : : : +- * ColumnarToRow (19) + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (18) + : : : : : : +- ReusedExchange (21) + : : : : : +- BroadcastExchange (27) + : : : : : +- * Filter (26) + : : : : : +- * ColumnarToRow (25) + : : : : : +- Scan parquet spark_catalog.default.item (24) + : : : : +- * Sort (52) + : : : : +- Exchange (51) + : : : : +- * Project (50) + : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : :- * Project (47) + : : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : : :- * Filter (44) + : : : : : : +- * ColumnarToRow (43) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (42) + : : : : : +- ReusedExchange (45) + : : : : +- ReusedExchange (48) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : :- * Filter (83) + : : +- * HashAggregate (82) + : : +- Exchange (81) + : : +- * HashAggregate (80) + : : +- * Project (79) + : : +- * BroadcastHashJoin Inner BuildRight (78) + : : :- * Project (76) + : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) + : : : : :- * Filter (71) + : : : : : +- * ColumnarToRow (70) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) + : : : : +- ReusedExchange (72) + : : : +- ReusedExchange (74) + : : +- ReusedExchange (77) + : +- * Filter (98) + : +- * HashAggregate (97) + : +- Exchange (96) + : +- * HashAggregate (95) + : +- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * Project (91) + : : +- * BroadcastHashJoin Inner BuildRight (90) + : : :- * BroadcastHashJoin LeftSemi BuildRight (88) + : : : :- * Filter (86) + : : : : +- * ColumnarToRow (85) + : : : : +- Scan parquet spark_catalog.default.web_sales (84) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (89) + : +- ReusedExchange (92) + :- * HashAggregate (107) + : +- Exchange (106) + : +- * HashAggregate (105) + : +- * HashAggregate (104) + : +- ReusedExchange (103) + :- * HashAggregate (112) + : +- Exchange (111) + : +- * HashAggregate (110) + : +- * HashAggregate (109) + : +- ReusedExchange (108) + :- * HashAggregate (117) + : +- Exchange (116) + : +- * HashAggregate (115) + : +- * HashAggregate (114) + : +- ReusedExchange (113) + +- * HashAggregate (122) + +- Exchange (121) + +- * HashAggregate (120) + +- * HashAggregate (119) + +- ReusedExchange (118) (1) Scan parquet spark_catalog.default.store_sales @@ -171,7 +172,7 @@ Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(10) ReusedExchange [Reuses operator id: 160] +(10) ReusedExchange [Reuses operator id: 161] Output [1]: [d_date_sk#13] (11) BroadcastHashJoin [codegen id : 9] @@ -221,7 +222,7 @@ Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] Condition : isnotnull(cs_item_sk#18) -(21) ReusedExchange [Reuses operator id: 160] +(21) ReusedExchange [Reuses operator id: 161] Output [1]: [d_date_sk#20] (22) BroadcastHashJoin [codegen id : 6] @@ -287,36 +288,40 @@ Join type: Inner Join condition: None (35) Project [codegen id : 9] -Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Output [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17] -(36) HashAggregate [codegen id : 9] +(36) Project [codegen id : 9] +Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27] +Input [3]: [i_brand_id#15, i_class_id#16, i_category_id#17] + +(37) HashAggregate [codegen id : 9] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(37) Exchange +(38) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [brand_id#25, class_id#26, category_id#27] Keys [3]: [brand_id#25, class_id#26, category_id#27] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#25, class_id#26, category_id#27] -(39) Exchange +(40) Exchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6] -(40) Sort [codegen id : 11] +(41) Sort [codegen id : 11] Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.web_sales +(42) Scan parquet spark_catalog.default.web_sales Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] @@ -324,126 +329,126 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 14] +(43) ColumnarToRow [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(43) Filter [codegen id : 14] +(44) Filter [codegen id : 14] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) -(44) ReusedExchange [Reuses operator id: 160] +(45) ReusedExchange [Reuses operator id: 161] Output [1]: [d_date_sk#30] -(45) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(46) Project [codegen id : 14] +(47) Project [codegen id : 14] Output [1]: [ws_item_sk#28] Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30] -(47) ReusedExchange [Reuses operator id: 27] +(48) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(48) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(49) Project [codegen id : 14] +(50) Project [codegen id : 14] Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34] -(50) Exchange +(51) Exchange Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) Sort [codegen id : 15] +(52) Sort [codegen id : 15] Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34] Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 16] +(53) SortMergeJoin [codegen id : 16] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)] Join type: LeftSemi Join condition: None -(53) BroadcastExchange +(54) BroadcastExchange Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8] -(54) BroadcastHashJoin [codegen id : 17] +(55) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(55) Project [codegen id : 17] +(56) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] -(56) BroadcastExchange +(57) BroadcastExchange Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(57) BroadcastHashJoin [codegen id : 20] +(58) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(58) ReusedExchange [Reuses operator id: 155] +(59) ReusedExchange [Reuses operator id: 156] Output [1]: [d_date_sk#36] -(59) BroadcastHashJoin [codegen id : 20] +(60) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(60) Project [codegen id : 20] +(61) Project [codegen id : 20] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36] -(61) ReusedExchange [Reuses operator id: 27] +(62) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(62) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#37] Join type: Inner Join condition: None -(63) Project [codegen id : 20] +(64) Project [codegen id : 20] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40] -(64) HashAggregate [codegen id : 20] +(65) HashAggregate [codegen id : 20] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] -(65) Exchange +(66) Exchange Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(66) HashAggregate [codegen id : 21] +(67) HashAggregate [codegen id : 21] Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46] Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] -(67) Filter [codegen id : 21] +(68) Filter [codegen id : 21] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(68) Scan parquet spark_catalog.default.catalog_sales +(69) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] @@ -451,71 +456,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 41] +(70) ColumnarToRow [codegen id : 41] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(70) Filter [codegen id : 41] +(71) Filter [codegen id : 41] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Condition : isnotnull(cs_item_sk#54) -(71) ReusedExchange [Reuses operator id: 56] +(72) ReusedExchange [Reuses operator id: 57] Output [1]: [ss_item_sk#35] -(72) BroadcastHashJoin [codegen id : 41] +(73) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(73) ReusedExchange [Reuses operator id: 155] +(74) ReusedExchange [Reuses operator id: 156] Output [1]: [d_date_sk#58] -(74) BroadcastHashJoin [codegen id : 41] +(75) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_sold_date_sk#57] Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(75) Project [codegen id : 41] +(76) Project [codegen id : 41] Output [3]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56] Input [5]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, d_date_sk#58] -(76) ReusedExchange [Reuses operator id: 27] +(77) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(77) BroadcastHashJoin [codegen id : 41] +(78) BroadcastHashJoin [codegen id : 41] Left keys [1]: [cs_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(78) Project [codegen id : 41] +(79) Project [codegen id : 41] Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(79) HashAggregate [codegen id : 41] +(80) HashAggregate [codegen id : 41] Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -(80) Exchange +(81) Exchange Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(81) HashAggregate [codegen id : 42] +(82) HashAggregate [codegen id : 42] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70] Results [6]: [catalog AS channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73] -(82) Filter [codegen id : 42] +(83) Filter [codegen id : 42] Input [6]: [channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(83) Scan parquet spark_catalog.default.web_sales +(84) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] @@ -523,443 +528,443 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 62] +(85) ColumnarToRow [codegen id : 62] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -(85) Filter [codegen id : 62] +(86) Filter [codegen id : 62] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Condition : isnotnull(ws_item_sk#74) -(86) ReusedExchange [Reuses operator id: 56] +(87) ReusedExchange [Reuses operator id: 57] Output [1]: [ss_item_sk#35] -(87) BroadcastHashJoin [codegen id : 62] +(88) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#74] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi Join condition: None -(88) ReusedExchange [Reuses operator id: 155] +(89) ReusedExchange [Reuses operator id: 156] Output [1]: [d_date_sk#78] -(89) BroadcastHashJoin [codegen id : 62] +(90) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(90) Project [codegen id : 62] +(91) Project [codegen id : 62] Output [3]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76] Input [5]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, d_date_sk#78] -(91) ReusedExchange [Reuses operator id: 27] +(92) ReusedExchange [Reuses operator id: 27] Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(92) BroadcastHashJoin [codegen id : 62] +(93) BroadcastHashJoin [codegen id : 62] Left keys [1]: [ws_item_sk#74] Right keys [1]: [i_item_sk#79] Join type: Inner Join condition: None -(93) Project [codegen id : 62] +(94) Project [codegen id : 62] Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82] Input [7]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(94) HashAggregate [codegen id : 62] +(95) HashAggregate [codegen id : 62] Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82] Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -(95) Exchange +(96) Exchange Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(96) HashAggregate [codegen id : 63] +(97) HashAggregate [codegen id : 63] Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90] Results [6]: [web AS channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93] -(97) Filter [codegen id : 63] +(98) Filter [codegen id : 63] Input [6]: [channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(98) Union +(99) Union -(99) HashAggregate [codegen id : 64] +(100) HashAggregate [codegen id : 64] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] Aggregate Attributes [3]: [sum#94, isEmpty#95, sum#96] Results [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(100) Exchange +(101) Exchange Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(101) HashAggregate [codegen id : 65] +(102) HashAggregate [codegen id : 65] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(102) ReusedExchange [Reuses operator id: 100] +(103) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(103) HashAggregate [codegen id : 130] +(104) HashAggregate [codegen id : 130] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [5]: [channel#49, i_brand_id#38, i_class_id#39, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(104) HashAggregate [codegen id : 130] +(105) HashAggregate [codegen id : 130] Input [5]: [channel#49, i_brand_id#38, i_class_id#39, sum_sales#102, number_sales#103] Keys [3]: [channel#49, i_brand_id#38, i_class_id#39] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109] -(105) Exchange +(106) Exchange Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109] Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(106) HashAggregate [codegen id : 131] +(107) HashAggregate [codegen id : 131] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109] Keys [3]: [channel#49, i_brand_id#38, i_class_id#39] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#110, sum(number_sales#103)#111] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, null AS i_category_id#112, sum(sum_sales#102)#110 AS sum(sum_sales)#113, sum(number_sales#103)#111 AS sum(number_sales)#114] -(107) ReusedExchange [Reuses operator id: 100] +(108) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(108) HashAggregate [codegen id : 196] +(109) HashAggregate [codegen id : 196] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [4]: [channel#49, i_brand_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(109) HashAggregate [codegen id : 196] +(110) HashAggregate [codegen id : 196] Input [4]: [channel#49, i_brand_id#38, sum_sales#102, number_sales#103] Keys [2]: [channel#49, i_brand_id#38] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] Results [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120] -(110) Exchange +(111) Exchange Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120] Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(111) HashAggregate [codegen id : 197] +(112) HashAggregate [codegen id : 197] Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120] Keys [2]: [channel#49, i_brand_id#38] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#121, sum(number_sales#103)#122] Results [6]: [channel#49, i_brand_id#38, null AS i_class_id#123, null AS i_category_id#124, sum(sum_sales#102)#121 AS sum(sum_sales)#125, sum(number_sales#103)#122 AS sum(number_sales)#126] -(112) ReusedExchange [Reuses operator id: 100] +(113) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(113) HashAggregate [codegen id : 262] +(114) HashAggregate [codegen id : 262] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(114) HashAggregate [codegen id : 262] +(115) HashAggregate [codegen id : 262] Input [3]: [channel#49, sum_sales#102, number_sales#103] Keys [1]: [channel#49] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129] Results [4]: [channel#49, sum#130, isEmpty#131, sum#132] -(115) Exchange +(116) Exchange Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(116) HashAggregate [codegen id : 263] +(117) HashAggregate [codegen id : 263] Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] Keys [1]: [channel#49] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#133, sum(number_sales#103)#134] Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null AS i_category_id#137, sum(sum_sales#102)#133 AS sum(sum_sales)#138, sum(number_sales#103)#134 AS sum(number_sales)#139] -(117) ReusedExchange [Reuses operator id: 100] +(118) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] -(118) HashAggregate [codegen id : 328] +(119) HashAggregate [codegen id : 328] Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(119) HashAggregate [codegen id : 328] +(120) HashAggregate [codegen id : 328] Input [2]: [sum_sales#102, number_sales#103] Keys: [] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] Aggregate Attributes [3]: [sum#140, isEmpty#141, sum#142] Results [3]: [sum#143, isEmpty#144, sum#145] -(120) Exchange +(121) Exchange Input [3]: [sum#143, isEmpty#144, sum#145] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] -(121) HashAggregate [codegen id : 329] +(122) HashAggregate [codegen id : 329] Input [3]: [sum#143, isEmpty#144, sum#145] Keys: [] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] Aggregate Attributes [2]: [sum(sum_sales#102)#146, sum(number_sales#103)#147] Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#150, null AS i_category_id#151, sum(sum_sales#102)#146 AS sum(sum_sales)#152, sum(number_sales#103)#147 AS sum(number_sales)#153] -(122) Union +(123) Union -(123) HashAggregate [codegen id : 330] +(124) HashAggregate [codegen id : 330] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] -(124) Exchange +(125) Exchange Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=18] -(125) HashAggregate [codegen id : 331] +(126) HashAggregate [codegen id : 331] Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Functions: [] Aggregate Attributes: [] Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] -(126) TakeOrderedAndProject +(127) TakeOrderedAndProject Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103] ===== Subqueries ===== -Subquery:1 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (145) -+- Exchange (144) - +- * HashAggregate (143) - +- Union (142) - :- * Project (131) - : +- * BroadcastHashJoin Inner BuildRight (130) - : :- * ColumnarToRow (128) - : : +- Scan parquet spark_catalog.default.store_sales (127) - : +- ReusedExchange (129) - :- * Project (136) - : +- * BroadcastHashJoin Inner BuildRight (135) - : :- * ColumnarToRow (133) - : : +- Scan parquet spark_catalog.default.catalog_sales (132) - : +- ReusedExchange (134) - +- * Project (141) - +- * BroadcastHashJoin Inner BuildRight (140) - :- * ColumnarToRow (138) - : +- Scan parquet spark_catalog.default.web_sales (137) - +- ReusedExchange (139) - - -(127) Scan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (146) ++- Exchange (145) + +- * HashAggregate (144) + +- Union (143) + :- * Project (132) + : +- * BroadcastHashJoin Inner BuildRight (131) + : :- * ColumnarToRow (129) + : : +- Scan parquet spark_catalog.default.store_sales (128) + : +- ReusedExchange (130) + :- * Project (137) + : +- * BroadcastHashJoin Inner BuildRight (136) + : :- * ColumnarToRow (134) + : : +- Scan parquet spark_catalog.default.catalog_sales (133) + : +- ReusedExchange (135) + +- * Project (142) + +- * BroadcastHashJoin Inner BuildRight (141) + :- * ColumnarToRow (139) + : +- Scan parquet spark_catalog.default.web_sales (138) + +- ReusedExchange (140) + + +(128) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 2] +(129) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] -(129) ReusedExchange [Reuses operator id: 160] +(130) ReusedExchange [Reuses operator id: 161] Output [1]: [d_date_sk#157] -(130) BroadcastHashJoin [codegen id : 2] +(131) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#156] Right keys [1]: [d_date_sk#157] Join type: Inner Join condition: None -(131) Project [codegen id : 2] +(132) Project [codegen id : 2] Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159] Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157] -(132) Scan parquet spark_catalog.default.catalog_sales +(133) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 4] +(134) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] -(134) ReusedExchange [Reuses operator id: 150] +(135) ReusedExchange [Reuses operator id: 151] Output [1]: [d_date_sk#164] -(135) BroadcastHashJoin [codegen id : 4] +(136) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#162] Right keys [1]: [d_date_sk#164] Join type: Inner Join condition: None -(136) Project [codegen id : 4] +(137) Project [codegen id : 4] Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166] Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164] -(137) Scan parquet spark_catalog.default.web_sales +(138) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 6] +(139) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] -(139) ReusedExchange [Reuses operator id: 150] +(140) ReusedExchange [Reuses operator id: 151] Output [1]: [d_date_sk#170] -(140) BroadcastHashJoin [codegen id : 6] +(141) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#169] Right keys [1]: [d_date_sk#170] Join type: Inner Join condition: None -(141) Project [codegen id : 6] +(142) Project [codegen id : 6] Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172] Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170] -(142) Union +(143) Union -(143) HashAggregate [codegen id : 7] +(144) HashAggregate [codegen id : 7] Input [2]: [quantity#158, list_price#159] Keys: [] Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] Aggregate Attributes [2]: [sum#173, count#174] Results [2]: [sum#175, count#176] -(144) Exchange +(145) Exchange Input [2]: [sum#175, count#176] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19] -(145) HashAggregate [codegen id : 8] +(146) HashAggregate [codegen id : 8] Input [2]: [sum#175, count#176] Keys: [] Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177] Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178] -Subquery:2 Hosting operator id = 127 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 128 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 -BroadcastExchange (150) -+- * Project (149) - +- * Filter (148) - +- * ColumnarToRow (147) - +- Scan parquet spark_catalog.default.date_dim (146) +Subquery:3 Hosting operator id = 133 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 +BroadcastExchange (151) ++- * Project (150) + +- * Filter (149) + +- * ColumnarToRow (148) + +- Scan parquet spark_catalog.default.date_dim (147) -(146) Scan parquet spark_catalog.default.date_dim +(147) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#164, d_year#179] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(147) ColumnarToRow [codegen id : 1] +(148) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#164, d_year#179] -(148) Filter [codegen id : 1] +(149) Filter [codegen id : 1] Input [2]: [d_date_sk#164, d_year#179] Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#164)) -(149) Project [codegen id : 1] +(150) Project [codegen id : 1] Output [1]: [d_date_sk#164] Input [2]: [d_date_sk#164, d_year#179] -(150) BroadcastExchange +(151) BroadcastExchange Input [1]: [d_date_sk#164] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] -Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 +Subquery:4 Hosting operator id = 138 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (155) -+- * Project (154) - +- * Filter (153) - +- * ColumnarToRow (152) - +- Scan parquet spark_catalog.default.date_dim (151) +BroadcastExchange (156) ++- * Project (155) + +- * Filter (154) + +- * ColumnarToRow (153) + +- Scan parquet spark_catalog.default.date_dim (152) -(151) Scan parquet spark_catalog.default.date_dim +(152) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#36, d_year#180, d_moy#181] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(152) ColumnarToRow [codegen id : 1] +(153) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#36, d_year#180, d_moy#181] -(153) Filter [codegen id : 1] +(154) Filter [codegen id : 1] Input [3]: [d_date_sk#36, d_year#180, d_moy#181] Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 = 2000)) AND (d_moy#181 = 11)) AND isnotnull(d_date_sk#36)) -(154) Project [codegen id : 1] +(155) Project [codegen id : 1] Output [1]: [d_date_sk#36] Input [3]: [d_date_sk#36, d_year#180, d_moy#181] -(155) BroadcastExchange +(156) BroadcastExchange Input [1]: [d_date_sk#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (160) -+- * Project (159) - +- * Filter (158) - +- * ColumnarToRow (157) - +- Scan parquet spark_catalog.default.date_dim (156) +BroadcastExchange (161) ++- * Project (160) + +- * Filter (159) + +- * ColumnarToRow (158) + +- Scan parquet spark_catalog.default.date_dim (157) -(156) Scan parquet spark_catalog.default.date_dim +(157) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#182] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(157) ColumnarToRow [codegen id : 1] +(158) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#182] -(158) Filter [codegen id : 1] +(159) Filter [codegen id : 1] Input [2]: [d_date_sk#13, d_year#182] Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#13)) -(159) Project [codegen id : 1] +(160) Project [codegen id : 1] Output [1]: [d_date_sk#13] Input [2]: [d_date_sk#13, d_year#182] -(160) BroadcastExchange +(161) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22] Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 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 f9287bdb7893b..1697d599f65b5 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 @@ -107,61 +107,62 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (9) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - 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] #9 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (3) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (2) - 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] - InputAdapter - WholeStageCodegen (7) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (5) - Filter [i_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + 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] #9 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (3) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (2) + 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] + InputAdapter + WholeStageCodegen (7) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (6) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter WholeStageCodegen (15) Sort [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 ed3595d8fc700..048cef69e6664 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 @@ -27,46 +27,46 @@ TakeOrderedAndProject (120) : : : : : +- * ColumnarToRow (5) : : : : : +- Scan parquet spark_catalog.default.item (4) : : : : +- BroadcastExchange (47) - : : : : +- * HashAggregate (46) - : : : : +- Exchange (45) - : : : : +- * HashAggregate (44) - : : : : +- * Project (43) - : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : :- * Project (40) - : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (38) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (37) - : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- BroadcastExchange (36) - : : : : : +- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (32) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : :- * Filter (29) - : : : : : : : +- * ColumnarToRow (28) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (27) - : : : : : : +- ReusedExchange (30) - : : : : : +- ReusedExchange (33) - : : : : +- ReusedExchange (41) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : : :- * HashAggregate (35) + : : : : : +- Exchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : : :- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Filter (38) + : : : : : : +- * ColumnarToRow (37) + : : : : : : +- Scan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (39) + : : : : +- ReusedExchange (42) : : : +- ReusedExchange (52) : : +- ReusedExchange (55) : :- * Filter (76) @@ -158,10 +158,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 9] +(8) ColumnarToRow [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 9] +(9) Filter [codegen id : 6] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -172,10 +172,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 7] +(11) ColumnarToRow [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 7] +(12) Filter [codegen id : 4] Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) @@ -239,121 +239,121 @@ Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 7] +(26) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join type: LeftSemi Join condition: None -(27) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#25), dynamicpruningexpression(ws_sold_date_sk#25 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(28) ColumnarToRow [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(29) Filter [codegen id : 6] -Input [2]: [ws_item_sk#24, ws_sold_date_sk#25] -Condition : isnotnull(ws_item_sk#24) +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +(30) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#24] (31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#24] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None (32) Project [codegen id : 6] -Output [4]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29] -Input [6]: [ws_item_sk#24, ws_sold_date_sk#25, i_item_sk#26, i_brand_id#27, i_class_id#28, i_category_id#29] +Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] -(33) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#30] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#25] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None +(34) Exchange +Input [3]: [brand_id#25, class_id#26, category_id#27] +Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(35) Project [codegen id : 6] -Output [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Input [5]: [ws_sold_date_sk#25, i_brand_id#27, i_class_id#28, i_category_id#29, d_date_sk#30] +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#25, class_id#26, category_id#27] +Keys [3]: [brand_id#25, class_id#26, category_id#27] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#25, class_id#26, category_id#27] -(36) BroadcastExchange -Input [3]: [i_brand_id#27, i_class_id#28, i_category_id#29] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=3] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct -(37) BroadcastHashJoin [codegen id : 7] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#27, 0), isnull(i_brand_id#27), coalesce(i_class_id#28, 0), isnull(i_class_id#28), coalesce(i_category_id#29, 0), isnull(i_category_id#29)] -Join type: LeftSemi -Join condition: None +(37) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(38) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(38) Filter [codegen id : 9] +Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] +Condition : isnotnull(ws_item_sk#28) -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#28] +Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] +Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#31] +(42) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#34] -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#31] +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#29] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(43) Project [codegen id : 9] -Output [3]: [i_brand_id#14 AS brand_id#32, i_class_id#15 AS class_id#33, i_category_id#16 AS category_id#34] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#31] - -(44) HashAggregate [codegen id : 9] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] -(45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(45) BroadcastExchange +Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(46) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] +Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] +Join type: LeftSemi +Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] (48) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None (49) Project [codegen id : 11] Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#32, class_id#33, category_id#34] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] (50) BroadcastExchange Input [1]: [ss_item_sk#35] @@ -899,30 +899,30 @@ BroadcastExchange (154) (150) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_year#182] +Output [2]: [d_date_sk#24, d_year#182] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (151) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#182] +Input [2]: [d_date_sk#24, d_year#182] (152) Filter [codegen id : 1] -Input [2]: [d_date_sk#31, d_year#182] -Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#24, d_year#182] +Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#24)) (153) Project [codegen id : 1] -Output [1]: [d_date_sk#31] -Input [2]: [d_date_sk#31, d_year#182] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#182] (154) BroadcastExchange -Input [1]: [d_date_sk#31] +Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 27 Hosting Expression = ws_sold_date_sk#25 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 Subquery:9 Hosting operator id = 76 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] 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 1c4835db87cf1..ef686cd46134e 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 @@ -94,31 +94,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #6 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - 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 - BroadcastExchange #9 - WholeStageCodegen (7) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + 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 + BroadcastExchange #9 + WholeStageCodegen (4) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow @@ -145,24 +145,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (6) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter diff --git a/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt b/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt index 837b3956ef61c..07a84e4cc34cd 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q20/explain.txt @@ -1,39 +1,40 @@ == Physical Plan == -* Sort (35) -+- Exchange (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin LeftSemi BuildRight (25) +* Sort (36) ++- Exchange (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (27) + : +- * BroadcastHashJoin LeftSemi BuildRight (26) : :- * Filter (3) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.supplier (1) - : +- BroadcastExchange (24) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * BroadcastHashJoin LeftSemi BuildRight (12) - : : :- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.partsupp (4) - : : +- BroadcastExchange (11) - : : +- * Project (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet spark_catalog.default.part (7) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet spark_catalog.default.lineitem (13) - +- BroadcastExchange (31) - +- * Project (30) - +- * Filter (29) - +- * ColumnarToRow (28) - +- Scan parquet spark_catalog.default.nation (27) + : +- BroadcastExchange (25) + : +- * Project (24) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (23) + : :- BroadcastExchange (14) + : : +- * Project (13) + : : +- * BroadcastHashJoin LeftSemi BuildRight (12) + : : :- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet spark_catalog.default.partsupp (4) + : : +- BroadcastExchange (11) + : : +- * Project (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet spark_catalog.default.part (7) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * Filter (17) + : +- * ColumnarToRow (16) + : +- Scan parquet spark_catalog.default.lineitem (15) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- * ColumnarToRow (29) + +- Scan parquet spark_catalog.default.nation (28) (1) Scan parquet spark_catalog.default.supplier @@ -57,10 +58,10 @@ Location [not included in comparison]/{warehouse_dir}/partsupp] PushedFilters: [IsNotNull(ps_partkey), IsNotNull(ps_availqty), IsNotNull(ps_suppkey)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 4] +(5) ColumnarToRow [codegen id : 2] Input [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] -(6) Filter [codegen id : 4] +(6) Filter [codegen id : 2] Input [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] Condition : ((isnotnull(ps_partkey#5) AND isnotnull(ps_availqty#7)) AND isnotnull(ps_suppkey#6)) @@ -86,117 +87,119 @@ Input [2]: [p_partkey#8, p_name#9] Input [1]: [p_partkey#8] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ps_partkey#5] Right keys [1]: [p_partkey#8] Join type: LeftSemi Join condition: None -(13) Scan parquet spark_catalog.default.lineitem +(13) Project [codegen id : 2] +Output [2]: [ps_suppkey#6, ps_availqty#7] +Input [3]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7] + +(14) BroadcastExchange +Input [2]: [ps_suppkey#6, ps_availqty#7] +Arguments: IdentityBroadcastMode, [plan_id=2] + +(15) Scan parquet spark_catalog.default.lineitem Output [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] Batched: true Location [not included in comparison]/{warehouse_dir}/lineitem] -PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] +PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 2] +(16) ColumnarToRow [codegen id : 3] Input [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] -(15) Filter [codegen id : 2] +(17) Filter [codegen id : 3] Input [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] -Condition : ((((isnotnull(l_shipdate#13) AND (l_shipdate#13 >= 1994-01-01)) AND (l_shipdate#13 < 1995-01-01)) AND isnotnull(l_partkey#10)) AND isnotnull(l_suppkey#11)) +Condition : ((((((isnotnull(l_partkey#10) AND isnotnull(l_suppkey#11)) AND isnotnull(l_shipdate#13)) AND (l_partkey#10 = outer(ps_partkey#5))) AND (l_suppkey#11 = outer(ps_suppkey#6))) AND (l_shipdate#13 >= 1994-01-01)) AND (l_shipdate#13 < 1995-01-01)) -(16) Project [codegen id : 2] -Output [3]: [l_partkey#10, l_suppkey#11, l_quantity#12] +(18) Project [codegen id : 3] +Output [1]: [l_quantity#12] Input [4]: [l_partkey#10, l_suppkey#11, l_quantity#12, l_shipdate#13] -(17) HashAggregate [codegen id : 2] -Input [3]: [l_partkey#10, l_suppkey#11, l_quantity#12] -Keys [2]: [l_partkey#10, l_suppkey#11] +(19) HashAggregate [codegen id : 3] +Input [1]: [l_quantity#12] +Keys: [] Functions [1]: [partial_sum(l_quantity#12)] Aggregate Attributes [2]: [sum#14, isEmpty#15] -Results [4]: [l_partkey#10, l_suppkey#11, sum#16, isEmpty#17] +Results [2]: [sum#16, isEmpty#17] -(18) Exchange -Input [4]: [l_partkey#10, l_suppkey#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(l_partkey#10, l_suppkey#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) Exchange +Input [2]: [sum#16, isEmpty#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(19) HashAggregate [codegen id : 3] -Input [4]: [l_partkey#10, l_suppkey#11, sum#16, isEmpty#17] -Keys [2]: [l_partkey#10, l_suppkey#11] +(21) HashAggregate +Input [2]: [sum#16, isEmpty#17] +Keys: [] Functions [1]: [sum(l_quantity#12)] Aggregate Attributes [1]: [sum(l_quantity#12)#18] -Results [3]: [(0.5 * sum(l_quantity#12)#18) AS (0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] +Results [1]: [(0.5 * sum(l_quantity#12)#18) AS (0.5 * sum(l_quantity))#19] -(20) Filter [codegen id : 3] -Input [3]: [(0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] +(22) Filter +Input [1]: [(0.5 * sum(l_quantity))#19] Condition : isnotnull((0.5 * sum(l_quantity))#19) -(21) BroadcastExchange -Input [3]: [(0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true], input[2, bigint, true]),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ps_partkey#5, ps_suppkey#6] -Right keys [2]: [l_partkey#10, l_suppkey#11] +(23) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: (cast(ps_availqty#7 as decimal(22,1)) > (0.5 * sum(l_quantity))#19) -(23) Project [codegen id : 4] +(24) Project [codegen id : 4] Output [1]: [ps_suppkey#6] -Input [6]: [ps_partkey#5, ps_suppkey#6, ps_availqty#7, (0.5 * sum(l_quantity))#19, l_partkey#10, l_suppkey#11] +Input [3]: [ps_suppkey#6, ps_availqty#7, (0.5 * sum(l_quantity))#19] -(24) BroadcastExchange +(25) BroadcastExchange Input [1]: [ps_suppkey#6] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_suppkey#1] Right keys [1]: [ps_suppkey#6] Join type: LeftSemi Join condition: None -(26) Project [codegen id : 6] +(27) Project [codegen id : 6] Output [3]: [s_name#2, s_address#3, s_nationkey#4] Input [4]: [s_suppkey#1, s_name#2, s_address#3, s_nationkey#4] -(27) Scan parquet spark_catalog.default.nation +(28) Scan parquet spark_catalog.default.nation Output [2]: [n_nationkey#20, n_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/nation] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 5] +(29) ColumnarToRow [codegen id : 5] Input [2]: [n_nationkey#20, n_name#21] -(29) Filter [codegen id : 5] +(30) Filter [codegen id : 5] Input [2]: [n_nationkey#20, n_name#21] Condition : ((isnotnull(n_name#21) AND (n_name#21 = CANADA)) AND isnotnull(n_nationkey#20)) -(30) Project [codegen id : 5] +(31) Project [codegen id : 5] Output [1]: [n_nationkey#20] Input [2]: [n_nationkey#20, n_name#21] -(31) BroadcastExchange +(32) BroadcastExchange Input [1]: [n_nationkey#20] Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_nationkey#4] Right keys [1]: [n_nationkey#20] Join type: Inner Join condition: None -(33) Project [codegen id : 6] +(34) Project [codegen id : 6] Output [2]: [s_name#2, s_address#3] Input [4]: [s_name#2, s_address#3, s_nationkey#4, n_nationkey#20] -(34) Exchange +(35) Exchange Input [2]: [s_name#2, s_address#3] Arguments: rangepartitioning(s_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(35) Sort [codegen id : 7] +(36) Sort [codegen id : 7] Input [2]: [s_name#2, s_address#3] Arguments: [s_name#2 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt b/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt index 65abd21e08684..4e54b2451d62c 100644 --- a/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt +++ b/sql/core/src/test/resources/tpch-plan-stability/q20/simplified.txt @@ -15,34 +15,35 @@ WholeStageCodegen (7) BroadcastExchange #2 WholeStageCodegen (4) Project [ps_suppkey] - BroadcastHashJoin [ps_partkey,ps_suppkey,l_partkey,l_suppkey,ps_availqty,(0.5 * sum(l_quantity))] - BroadcastHashJoin [ps_partkey,p_partkey] - Filter [ps_partkey,ps_availqty,ps_suppkey] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.partsupp [ps_partkey,ps_suppkey,ps_availqty] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [p_partkey] - Filter [p_name,p_partkey] + BroadcastNestedLoopJoin [ps_availqty,(0.5 * sum(l_quantity))] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [ps_suppkey,ps_availqty] + BroadcastHashJoin [ps_partkey,p_partkey] + Filter [ps_partkey,ps_availqty,ps_suppkey] ColumnarToRow InputAdapter - Scan parquet spark_catalog.default.part [p_partkey,p_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [(0.5 * sum(l_quantity))] - HashAggregate [l_partkey,l_suppkey,sum,isEmpty] [sum(l_quantity),(0.5 * sum(l_quantity)),sum,isEmpty] + Scan parquet spark_catalog.default.partsupp [ps_partkey,ps_suppkey,ps_availqty] InputAdapter - Exchange [l_partkey,l_suppkey] #5 - WholeStageCodegen (2) - HashAggregate [l_partkey,l_suppkey,l_quantity] [sum,isEmpty,sum,isEmpty] - Project [l_partkey,l_suppkey,l_quantity] - Filter [l_shipdate,l_partkey,l_suppkey] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.lineitem [l_partkey,l_suppkey,l_quantity,l_shipdate] + BroadcastExchange #4 + WholeStageCodegen (1) + Project [p_partkey] + Filter [p_name,p_partkey] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.part [p_partkey,p_name] + Filter [(0.5 * sum(l_quantity))] + HashAggregate [sum,isEmpty] [sum(l_quantity),(0.5 * sum(l_quantity)),sum,isEmpty] + InputAdapter + Exchange #5 + WholeStageCodegen (3) + HashAggregate [l_quantity] [sum,isEmpty,sum,isEmpty] + Project [l_quantity] + Filter [l_partkey,l_suppkey,l_shipdate] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.lineitem [l_partkey,l_suppkey,l_quantity,l_shipdate] InputAdapter BroadcastExchange #6 WholeStageCodegen (5)