diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 11b675e75869e..c41686da79487 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -349,14 +349,11 @@ object JoinReorderDP extends PredicateHelper with Logging { } def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { - if (other.planCost.card == 0 || other.planCost.size == 0) { - false - } else { - val relativeRows = BigDecimal(this.planCost.card) / BigDecimal(other.planCost.card) - val relativeSize = BigDecimal(this.planCost.size) / BigDecimal(other.planCost.size) - relativeRows * conf.joinReorderCardWeight + - relativeSize * (1 - conf.joinReorderCardWeight) < 1 - } + val thisCost = BigDecimal(this.planCost.card) * conf.joinReorderCardWeight + + BigDecimal(this.planCost.size) * (1 - conf.joinReorderCardWeight) + val otherCost = BigDecimal(other.planCost.card) * conf.joinReorderCardWeight + + BigDecimal(other.planCost.size) * (1 - conf.joinReorderCardWeight) + thisCost < otherCost } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index b84207397e5cc..2e1cf4a137e25 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.optimizer.JoinReorderDP.JoinPlan import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -363,4 +364,18 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB assertEqualJoinPlans(Optimize, originalPlan3, bestPlan3) } + + test("SPARK-33935: betterThan should be consistent") { + val plan1 = JoinPlan(null, null, null, Cost(300, 80)) + val plan2 = JoinPlan(null, null, null, Cost(500, 30)) + + // cost1 = 300*0.7 + 80*0.3 = 234 + // cost2 = 500*0.7 + 30*0.3 = 359 + + assert(!plan1.betterThan(plan1, conf)) + assert(!plan2.betterThan(plan2, conf)) + + assert(plan1.betterThan(plan2, conf)) + assert(!plan2.betterThan(plan1, conf)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala index 703be48c6a2a9..a42914765dcc8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala @@ -294,12 +294,12 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs (nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) val expected = - f1.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) - .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) - .join(t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))), Inner, - Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) + t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) + .join(f1 + .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) + .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk")))) .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualJoinPlans(Optimize, query, expected) 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 8ee427262b332..327e7db702faa 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 @@ -4,8 +4,8 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) : :- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (15) @@ -27,16 +27,16 @@ : : +- * Project (19) : : +- * Filter (18) : : +- * ColumnarToRow (17) - : : +- Scan parquet default.date_dim (16) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.store (23) + : : +- Scan parquet default.customer_address (16) + : +- BroadcastExchange (27) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.date_dim (23) +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer_address (29) + +- * Filter (32) + +- * ColumnarToRow (31) + +- Scan parquet default.store (30) (1) Scan parquet default.store_sales @@ -107,94 +107,94 @@ Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = A Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_year#19] +(16) Scan parquet default.customer_address +Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] 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 +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, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] +ReadSchema: struct (17) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#18, d_year#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] (18) Filter [codegen id : 3] -Input [2]: [d_date_sk#18, d_year#19] -Condition : ((isnotnull(d_year#19) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) +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))) (19) Project [codegen id : 3] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_year#19] +Output [2]: [ca_address_sk#18, ca_state#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] (20) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None +Left keys [1]: [ss_addr_sk#4] +Right keys [1]: [ca_address_sk#18] +Join condition: ((((ca_state#19 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#19 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#19 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) (22) Project [codegen id : 6] -Output [6]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [8]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, d_date_sk#18] +Output [5]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [9]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#18, ca_state#19] -(23) Scan parquet default.store -Output [1]: [s_store_sk#21] +(23) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [1]: [s_store_sk#21] +Input [2]: [d_date_sk#22, d_year#23] (25) Filter [codegen id : 4] -Input [1]: [s_store_sk#21] -Condition : isnotnull(s_store_sk#21) +Input [2]: [d_date_sk#22, d_year#23] +Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(26) BroadcastExchange -Input [1]: [s_store_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(26) Project [codegen id : 4] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#21] +(27) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] Join condition: None -(28) Project [codegen id : 6] -Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] +(29) Project [codegen id : 6] +Output [4]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [6]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, d_date_sk#22] -(29) Scan parquet default.customer_address -Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +(30) Scan parquet default.store +Output [1]: [s_store_sk#25] 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, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct -(31) Filter [codegen id : 5] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] -Condition : (((isnotnull(ca_country#25) AND (ca_country#25 = United States)) AND isnotnull(ca_address_sk#23)) AND ((ca_state#24 IN (TX,OH) OR ca_state#24 IN (OR,NM,KY)) OR ca_state#24 IN (VA,TX,MS))) +(31) ColumnarToRow [codegen id : 5] +Input [1]: [s_store_sk#25] -(32) Project [codegen id : 5] -Output [2]: [ca_address_sk#23, ca_state#24] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +(32) Filter [codegen id : 5] +Input [1]: [s_store_sk#25] +Condition : isnotnull(s_store_sk#25) (33) BroadcastExchange -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [s_store_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#23] -Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#25] +Join condition: None (35) Project [codegen id : 6] Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] +Input [5]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, s_store_sk#25] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] @@ -205,7 +205,7 @@ Results [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] (37) Exchange Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] -Arguments: SinglePartition, true, [id=#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#41] (38) HashAggregate [codegen id : 7] Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] 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 b457788dbd0b2..45d6c8f3b0bae 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 @@ -5,11 +5,11 @@ 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_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] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] Project [ss_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] Project [ss_sold_date_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,cd_marital_status,cd_education_status] @@ -35,23 +35,23 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [ca_address_sk,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index a17356ae04a03..a9ab8c3690a00 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * SortMergeJoin Inner (24) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store (11) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.item (19) - : +- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) + :- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.catalog_sales (44) - +- ReusedExchange (47) + +- * SortMergeJoin Inner (48) + :- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) + +- * Sort (47) + +- Exchange (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet default.catalog_sales (40) + +- ReusedExchange (43) (1) Scan parquet default.store_sales @@ -132,7 +132,7 @@ Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, s (17) Exchange Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (18) Sort [codegen id : 4] Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11] @@ -154,7 +154,7 @@ Condition : isnotnull(i_item_sk#14) (22) Exchange Input [3]: [i_item_sk#14, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(i_item_sk#14, 5), true, [id=#17] +Arguments: hashpartitioning(i_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#17] (23) Sort [codegen id : 6] Input [3]: [i_item_sk#14, i_item_id#15, i_item_desc#16] @@ -171,7 +171,7 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s (26) Exchange Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#18] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#18] (27) Sort [codegen id : 8] Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] @@ -224,89 +224,89 @@ Input [6]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), true, [id=#27] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#27] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 - -(40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] -Join condition: None - -(41) Project [codegen id : 12] -Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] -Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] - -(42) Exchange -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] -Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), true, [id=#28] - -(43) Sort [codegen id : 13] -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +(40) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 15] -Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +(41) ColumnarToRow [codegen id : 13] +Input [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] + +(42) Filter [codegen id : 13] +Input [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Condition : ((isnotnull(cs_bill_customer_sk#29) AND isnotnull(cs_item_sk#30)) AND isnotnull(cs_sold_date_sk#28)) -(46) Filter [codegen id : 15] -Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Condition : ((isnotnull(cs_bill_customer_sk#30) AND isnotnull(cs_item_sk#31)) AND isnotnull(cs_sold_date_sk#29)) +(43) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#32] -(47) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#33] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#28] +Right keys [1]: [d_date_sk#32] +Join condition: None + +(45) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Input [5]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31, d_date_sk#32] + +(46) Exchange +Input [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#29 as bigint), cast(cs_item_sk#30 as bigint), 5), ENSURE_REQUIREMENTS, [id=#33] -(48) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#29] -Right keys [1]: [d_date_sk#33] +(47) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Arguments: [cast(cs_bill_customer_sk#29 as bigint) ASC NULLS FIRST, cast(cs_item_sk#30 as bigint) ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] +Right keys [2]: [cast(cs_bill_customer_sk#29 as bigint), cast(cs_item_sk#30 as bigint)] Join condition: None (49) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Input [5]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32, d_date_sk#33] +Output [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Input [7]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] (50) Exchange -Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint), 5), true, [id=#34] +Input [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Arguments: [cast(cs_bill_customer_sk#30 as bigint) ASC NULLS FIRST, cast(cs_item_sk#31 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] -Right keys [2]: [cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint)] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] Join condition: None (53) Project [codegen id : 17] -Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] -Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#31, s_state#11, i_item_id#15, i_item_desc#16] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] (54) HashAggregate [codegen id : 17] -Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#31, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#31), partial_avg(cast(cs_quantity#31 as bigint)), partial_stddev_samp(cast(cs_quantity#31 as double))] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] (55) Exchange Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] -Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, [id=#71] +Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), ENSURE_REQUIREMENTS, [id=#71] (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#31), avg(cast(cs_quantity#31 as bigint)), stddev_samp(cast(cs_quantity#31 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#31)#78, avg(cast(cs_quantity#31 as bigint))#79, stddev_samp(cast(cs_quantity#31 as double))#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#31)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#31 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#31 as double))#80 / avg(cast(cs_quantity#31 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#31 as double))#80 / avg(cast(cs_quantity#31 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index bfb59441f483b..79226a34e6768 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + Scan parquet default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_state] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (16) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 + WholeStageCodegen (15) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,cs_quantity] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 + Exchange [sr_customer_sk,sr_item_sk] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - WholeStageCodegen (16) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter - ReusedExchange [d_date_sk] #9 + WholeStageCodegen (14) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (13) + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] + InputAdapter + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 516f782057631..12e95ba50cd0d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -34,24 +34,24 @@ TakeOrderedAndProject (53) +- * Sort (46) +- Exchange (45) +- * Project (44) - +- * SortMergeJoin Inner (43) - :- * Sort (37) - : +- Exchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet default.customer (26) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.customer_address (30) - +- * Sort (42) - +- Exchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (38) + : +- * SortMergeJoin Inner (37) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet default.customer (26) + : +- * Sort (36) + : +- Exchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_demographics (32) + +- BroadcastExchange (42) + +- * Filter (41) + +- * ColumnarToRow (40) + +- Scan parquet default.customer_address (39) (1) Scan parquet default.catalog_sales @@ -159,7 +159,7 @@ Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6 (24) Exchange Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] (25) Sort [codegen id : 5] Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -172,89 +172,89 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,6,8,9,12,2]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(27) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] Condition : (((c_birth_month#25 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) -(29) Project [codegen id : 7] +(29) Project [codegen id : 6] Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(30) Scan parquet default.customer_address -Output [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(32) Filter [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#27)) - -(33) BroadcastExchange -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] -Join condition: None - -(35) Project [codegen id : 7] -Output [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [8]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(36) Exchange -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#32] +(30) Exchange +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(37) Sort [codegen id : 8] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +(31) Sort [codegen id : 7] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#33] +(32) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 9] -Input [1]: [cd_demo_sk#33] +(33) ColumnarToRow [codegen id : 8] +Input [1]: [cd_demo_sk#28] -(40) Filter [codegen id : 9] -Input [1]: [cd_demo_sk#33] -Condition : isnotnull(cd_demo_sk#33) +(34) Filter [codegen id : 8] +Input [1]: [cd_demo_sk#28] +Condition : isnotnull(cd_demo_sk#28) -(41) Exchange -Input [1]: [cd_demo_sk#33] -Arguments: hashpartitioning(cd_demo_sk#33, 5), true, [id=#34] +(35) Exchange +Input [1]: [cd_demo_sk#28] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(42) Sort [codegen id : 10] -Input [1]: [cd_demo_sk#33] -Arguments: [cd_demo_sk#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 9] +Input [1]: [cd_demo_sk#28] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 11] +(37) SortMergeJoin [codegen id : 11] Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#33] +Right keys [1]: [cd_demo_sk#28] +Join condition: None + +(38) Project [codegen id : 11] +Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] + +(39) Scan parquet default.customer_address +Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(40) ColumnarToRow [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] + +(41) Filter [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#30)) + +(42) BroadcastExchange +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#24] +Right keys [1]: [ca_address_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30, cd_demo_sk#33] +Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#35] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -263,12 +263,12 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (48) Project [codegen id : 13] -Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] +Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] (49) Expand [codegen id : 13] -Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28] -Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] +Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] +Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] (50) HashAggregate [codegen id : 13] Input [12]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] @@ -279,7 +279,7 @@ Results [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_gro (51) Exchange Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), true, [id=#69] +Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), ENSURE_REQUIREMENTS, [id=#69] (52) HashAggregate [codegen id : 14] Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt index 8c76e7cab3310..8069d43c3451a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt @@ -54,34 +54,34 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag 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] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,c_birth_year] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (6) 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 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 default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (9) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #8 + WholeStageCodegen (8) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt index 88b5168f6049c..4627bc19f25f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt @@ -4,248 +4,248 @@ TakeOrderedAndProject (45) +- Exchange (43) +- * HashAggregate (42) +- * Project (41) - +- * SortMergeJoin Inner (40) - :- * Sort (25) - : +- Exchange (24) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.store (18) - +- * Sort (39) - +- Exchange (38) - +- * Project (37) - +- * SortMergeJoin Inner (36) - :- * Sort (30) - : +- Exchange (29) - : +- * Filter (28) - : +- * ColumnarToRow (27) - : +- Scan parquet default.customer (26) - +- * Sort (35) - +- Exchange (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet default.customer_address (31) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (34) + : +- * SortMergeJoin Inner (33) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : :- BroadcastExchange (5) + : : : : +- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.date_dim (1) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.store_sales (6) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (32) + : +- Exchange (31) + : +- * Project (30) + : +- * SortMergeJoin Inner (29) + : :- * Sort (23) + : : +- Exchange (22) + : : +- * Filter (21) + : : +- * ColumnarToRow (20) + : : +- Scan parquet default.customer (19) + : +- * Sort (28) + : +- Exchange (27) + : +- * Filter (26) + : +- * ColumnarToRow (25) + : +- Scan parquet default.customer_address (24) + +- BroadcastExchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.item (35) + + +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 8)) AND isnotnull(i_item_sk#6)) +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(8) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#6] -Join condition: None +(4) Project [codegen id : 1] +Output [1]: [d_date_sk#1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(10) Project [codegen id : 4] -Output [8]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [10]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +(5) BroadcastExchange +Input [1]: [d_date_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +(6) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] - -(13) Filter [codegen id : 2] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_moy#15) AND isnotnull(d_year#14)) AND (d_moy#15 = 11)) AND (d_year#14 = 1998)) AND isnotnull(d_date_sk#13)) +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#13] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +(7) ColumnarToRow +Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -(15) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(8) Filter +Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +Condition : (((isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) AND isnotnull(ss_customer_sk#7)) AND isnotnull(ss_store_sk#8)) -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#13] +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None -(17) Project [codegen id : 4] -Output [7]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, d_date_sk#13] +(10) Project [codegen id : 3] +Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +Input [6]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -(18) Scan parquet default.store -Output [2]: [s_store_sk#17, s_zip#18] +(11) Scan parquet default.store +Output [2]: [s_store_sk#10, s_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#17, s_zip#18] +(12) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#10, s_zip#11] -(20) Filter [codegen id : 3] -Input [2]: [s_store_sk#17, s_zip#18] -Condition : (isnotnull(s_zip#18) AND isnotnull(s_store_sk#17)) +(13) Filter [codegen id : 2] +Input [2]: [s_store_sk#10, s_zip#11] +Condition : (isnotnull(s_zip#11) AND isnotnull(s_store_sk#10)) -(21) BroadcastExchange -Input [2]: [s_store_sk#17, s_zip#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] +(14) BroadcastExchange +Input [2]: [s_store_sk#10, s_zip#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#17] +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#8] +Right keys [1]: [s_store_sk#10] Join condition: None -(23) Project [codegen id : 4] -Output [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Input [9]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_store_sk#17, s_zip#18] +(16) Project [codegen id : 3] +Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9, s_store_sk#10, s_zip#11] -(24) Exchange -Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Arguments: hashpartitioning(ss_customer_sk#3, 5), true, [id=#20] +(17) Exchange +Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#13] -(25) Sort [codegen id : 5] -Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Arguments: [ss_customer_sk#3 ASC NULLS FIRST], false, 0 +(18) Sort [codegen id : 4] +Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(26) Scan parquet default.customer -Output [2]: [c_customer_sk#21, c_current_addr_sk#22] +(19) Scan parquet default.customer +Output [2]: [c_customer_sk#14, c_current_addr_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +(20) ColumnarToRow [codegen id : 5] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -(28) Filter [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) +(21) Filter [codegen id : 5] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) -(29) Exchange -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: hashpartitioning(c_current_addr_sk#22, 5), true, [id=#23] +(22) Exchange +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] -(30) Sort [codegen id : 7] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 6] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 -(31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_zip#25] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#17, ca_zip#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_zip#25] +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#17, ca_zip#18] -(33) Filter [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_zip#25] -Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) +(26) Filter [codegen id : 7] +Input [2]: [ca_address_sk#17, ca_zip#18] +Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) -(34) Exchange -Input [2]: [ca_address_sk#24, ca_zip#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), true, [id=#26] +(27) Exchange +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: hashpartitioning(ca_address_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] -(35) Sort [codegen id : 9] -Input [2]: [ca_address_sk#24, ca_zip#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin [codegen id : 10] -Left keys [1]: [c_current_addr_sk#22] -Right keys [1]: [ca_address_sk#24] +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#15] +Right keys [1]: [ca_address_sk#17] Join condition: None -(37) Project [codegen id : 10] -Output [2]: [c_customer_sk#21, ca_zip#25] -Input [4]: [c_customer_sk#21, c_current_addr_sk#22, ca_address_sk#24, ca_zip#25] +(30) Project [codegen id : 9] +Output [2]: [c_customer_sk#14, ca_zip#18] +Input [4]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_zip#18] + +(31) Exchange +Input [2]: [c_customer_sk#14, ca_zip#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] + +(32) Sort [codegen id : 10] +Input [2]: [c_customer_sk#14, ca_zip#18] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 -(38) Exchange -Input [2]: [c_customer_sk#21, ca_zip#25] -Arguments: hashpartitioning(c_customer_sk#21, 5), true, [id=#27] +(33) SortMergeJoin [codegen id : 12] +Left keys [1]: [ss_customer_sk#7] +Right keys [1]: [c_customer_sk#14] +Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#11, 1, 5)) -(39) Sort [codegen id : 11] -Input [2]: [c_customer_sk#21, ca_zip#25] -Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 +(34) Project [codegen id : 12] +Output [2]: [ss_item_sk#6, ss_ext_sales_price#9] +Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11, c_customer_sk#14, ca_zip#18] -(40) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#21] -Join condition: NOT (substr(ca_zip#25, 1, 5) = substr(s_zip#18, 1, 5)) +(35) Scan parquet default.item +Output [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct + +(36) ColumnarToRow [codegen id : 11] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] + +(37) Filter [codegen id : 11] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] +Condition : ((isnotnull(i_manager_id#26) AND (i_manager_id#26 = 8)) AND isnotnull(i_item_sk#21)) + +(38) Project [codegen id : 11] +Output [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] + +(39) BroadcastExchange +Input [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] + +(40) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#21] +Join condition: None (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [9]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18, c_customer_sk#21, ca_zip#25] +Output [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Input [7]: [ss_item_sk#6, ss_ext_sales_price#9, i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Input [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#9))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Results [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] (43) Exchange -Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] -Arguments: hashpartitioning(i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, 5), true, [id=#30] +Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Arguments: hashpartitioning(i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, 5), ENSURE_REQUIREMENTS, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] -Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#31] -Results [5]: [i_brand_id#7 AS brand_id#32, i_brand#8 AS brand#33, i_manufact_id#9, i_manufact#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS ext_price#34] +Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#9))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#9))#31] +Results [5]: [i_brand_id#22 AS brand_id#32, i_brand#23 AS brand#33, i_manufact_id#24, i_manufact#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#9))#31,17,2) AS ext_price#34] (45) TakeOrderedAndProject -Input [5]: [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] -Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#9 ASC NULLS FIRST, i_manufact#10 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] +Input [5]: [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] +Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#24 ASC NULLS FIRST, i_manufact#25 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt index 05fa3f82e27df..b6441c5fe72c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -6,71 +6,71 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (12) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] - InputAdapter - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #2 - WholeStageCodegen (4) - Project [ss_customer_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] + InputAdapter + WholeStageCodegen (4) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #2 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_zip,s_store_sk] - ColumnarToRow + Filter [s_zip,s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_zip] + InputAdapter + WholeStageCodegen (10) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #5 + WholeStageCodegen (9) + Project [c_customer_sk,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (6) + Sort [c_current_addr_sk] + InputAdapter + Exchange [c_current_addr_sk] #6 + WholeStageCodegen (5) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (8) + Sort [ca_address_sk] InputAdapter - Scan parquet default.store [s_store_sk,s_zip] + Exchange [ca_address_sk] #7 + WholeStageCodegen (7) + Filter [ca_address_sk,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_zip] InputAdapter - WholeStageCodegen (11) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (10) - Project [c_customer_sk,ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (7) - Sort [c_current_addr_sk] - InputAdapter - Exchange [c_current_addr_sk] #7 - WholeStageCodegen (6) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #8 - WholeStageCodegen (8) - Filter [ca_address_sk,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] + BroadcastExchange #8 + WholeStageCodegen (11) + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index ffcf6bd4f6d47..093c4eed6cf11 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -92,7 +92,7 @@ Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, (10) Exchange Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 3] Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] @@ -114,7 +114,7 @@ Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (15) Exchange Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] (16) Sort [codegen id : 5] Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] @@ -189,7 +189,7 @@ Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_c (32) Exchange Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#29] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] (33) Sort [codegen id : 9] Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] @@ -211,7 +211,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (37) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#32] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (38) Sort [codegen id : 11] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -235,7 +235,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (42) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#35] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 13] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] @@ -253,7 +253,7 @@ Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty# (45) Exchange Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), true, [id=#42] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] (46) HashAggregate [codegen id : 14] Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] @@ -309,12 +309,12 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (64) : : : +- * Filter (63) : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.item (61) + : : : +- Scan parquet default.customer (61) : : +- * Sort (74) : : +- Exchange (73) : : +- * Filter (72) : : +- * ColumnarToRow (71) - : : +- Scan parquet default.customer (70) + : : +- Scan parquet default.item (70) : +- * Sort (83) : +- Exchange (82) : +- * Filter (81) @@ -374,88 +374,88 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s (59) Exchange Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] (60) Sort [codegen id : 3] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(61) Scan parquet default.customer +Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct (62) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (63) Filter [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (64) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#50] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 5] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#14] Join condition: None (67) Project [codegen id : 6] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (68) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Scan parquet default.item +Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct (71) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (72) Filter [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Condition : isnotnull(i_item_sk#6) (73) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#52] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] (74) Sort [codegen id : 9] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] Join condition: None (76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -474,7 +474,7 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -513,7 +513,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (91) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] (92) Sort [codegen id : 17] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] @@ -537,7 +537,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (96) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#59] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] (97) HashAggregate [codegen id : 19] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] @@ -555,7 +555,7 @@ Results [2]: [sum#63, count#64] (99) Exchange Input [2]: [sum#63, count#64] -Arguments: SinglePartition, true, [id=#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (100) HashAggregate [codegen id : 20] Input [2]: [sum#63, count#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 10f874f8f5543..7de562c5d59a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #14 + Exchange [ss_item_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #15 + Exchange [ss_customer_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #17 + Exchange [c_customer_sk] #17 WholeStageCodegen (4) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (9) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #18 + Exchange [i_item_sk] #18 WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 73f36e3a9ca23..273950bed3546 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -92,7 +92,7 @@ Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, (10) Exchange Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 3] Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] @@ -114,7 +114,7 @@ Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (15) Exchange Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] (16) Sort [codegen id : 5] Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] @@ -189,7 +189,7 @@ Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_c (32) Exchange Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#29] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] (33) Sort [codegen id : 9] Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] @@ -211,7 +211,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (37) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#32] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (38) Sort [codegen id : 11] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -235,7 +235,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (42) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#35] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 13] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] @@ -253,7 +253,7 @@ Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty# (45) Exchange Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), true, [id=#42] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] (46) HashAggregate [codegen id : 14] Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] @@ -309,12 +309,12 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (64) : : : +- * Filter (63) : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.item (61) + : : : +- Scan parquet default.customer (61) : : +- * Sort (74) : : +- Exchange (73) : : +- * Filter (72) : : +- * ColumnarToRow (71) - : : +- Scan parquet default.customer (70) + : : +- Scan parquet default.item (70) : +- * Sort (83) : +- Exchange (82) : +- * Filter (81) @@ -374,88 +374,88 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s (59) Exchange Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] (60) Sort [codegen id : 3] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(61) Scan parquet default.customer +Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct (62) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (63) Filter [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (64) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#50] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 5] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#14] Join condition: None (67) Project [codegen id : 6] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (68) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Scan parquet default.item +Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct (71) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (72) Filter [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Condition : isnotnull(i_item_sk#6) (73) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#52] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] (74) Sort [codegen id : 9] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] Join condition: None (76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -474,7 +474,7 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -513,7 +513,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (91) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] (92) Sort [codegen id : 17] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] @@ -537,7 +537,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (96) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#59] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] (97) HashAggregate [codegen id : 19] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] @@ -555,7 +555,7 @@ Results [2]: [sum#63, count#64] (99) Exchange Input [2]: [sum#63, count#64] -Arguments: SinglePartition, true, [id=#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (100) HashAggregate [codegen id : 20] Input [2]: [sum#63, count#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 10f874f8f5543..7de562c5d59a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #14 + Exchange [ss_item_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #15 + Exchange [ss_customer_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #17 + Exchange [c_customer_sk] #17 WholeStageCodegen (4) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (9) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #18 + Exchange [i_item_sk] #18 WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index c6dc3db869003..3100e574e60e3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * SortMergeJoin Inner (24) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store (11) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.item (19) - : +- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) + :- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.catalog_sales (44) - +- ReusedExchange (47) + +- * SortMergeJoin Inner (48) + :- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) + +- * Sort (47) + +- Exchange (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet default.catalog_sales (40) + +- ReusedExchange (43) (1) Scan parquet default.store_sales @@ -132,7 +132,7 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, s (17) Exchange Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#15] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#15] (18) Sort [codegen id : 4] Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13] @@ -154,7 +154,7 @@ Condition : isnotnull(i_item_sk#16) (22) Exchange Input [3]: [i_item_sk#16, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(i_item_sk#16, 5), true, [id=#19] +Arguments: hashpartitioning(i_item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 6] Input [3]: [i_item_sk#16, i_item_id#17, i_item_desc#18] @@ -171,7 +171,7 @@ Input [9]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, (26) Exchange Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#20] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (27) Sort [codegen id : 8] Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] @@ -224,89 +224,89 @@ Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 - -(40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] -Join condition: None - -(41) Project [codegen id : 12] -Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] -Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] - -(42) Exchange -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] - -(43) Sort [codegen id : 13] -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +(40) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 15] -Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +(41) ColumnarToRow [codegen id : 13] +Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] + +(42) Filter [codegen id : 13] +Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Condition : ((isnotnull(cs_bill_customer_sk#32) AND isnotnull(cs_item_sk#33)) AND isnotnull(cs_sold_date_sk#31)) -(46) Filter [codegen id : 15] -Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Condition : ((isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) AND isnotnull(cs_sold_date_sk#32)) +(43) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#35] -(47) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#36] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#31] +Right keys [1]: [d_date_sk#35] +Join condition: None + +(45) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Input [5]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34, d_date_sk#35] + +(46) Exchange +Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#36] -(48) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#32] -Right keys [1]: [d_date_sk#36] +(47) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Arguments: [cast(cs_bill_customer_sk#32 as bigint) ASC NULLS FIRST, cast(cs_item_sk#33 as bigint) ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] +Right keys [2]: [cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint)] Join condition: None (49) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Input [5]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, d_date_sk#36] +Output [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Input [7]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] (50) Exchange -Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint), 5), true, [id=#37] +Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#37] (51) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Arguments: [cast(cs_bill_customer_sk#33 as bigint) ASC NULLS FIRST, cast(cs_item_sk#34 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] -Right keys [2]: [cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint)] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [13]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#34))] Aggregate Attributes [3]: [sum#38, sum#39, sum#40] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] (55) Exchange Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] -Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, 5), true, [id=#44] +Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#35))#47] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#34))#47] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#47,17,2) AS catalog_sales_profit#50] (57) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_profit#48, store_returns_loss#49, catalog_sales_profit#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index ad9fa718ff2bd..9b53cdaa5dc67 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (16) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 + WholeStageCodegen (15) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,cs_net_profit] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 + Exchange [sr_customer_sk,sr_item_sk] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - WholeStageCodegen (16) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) - Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter - ReusedExchange [d_date_sk] #9 + WholeStageCodegen (14) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (13) + Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] + InputAdapter + ReusedExchange [d_date_sk] #9 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 8185680b58670..cb8522545f1d3 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 @@ -9,8 +9,8 @@ TakeOrderedAndProject (67) : +- * HashAggregate (30) : +- * Project (29) : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) : : :- * Project (10) : : : +- * BroadcastHashJoin Inner BuildRight (9) : : : :- * Filter (3) @@ -21,21 +21,21 @@ TakeOrderedAndProject (67) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (20) - : : +- * BroadcastHashJoin LeftSemi BuildRight (19) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.item (11) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.item (14) + : : +- BroadcastExchange (15) + : : +- * Project (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer_address (11) : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.customer_address (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : :- * Filter (20) + : : +- * ColumnarToRow (19) + : : +- Scan parquet default.item (18) + : +- BroadcastExchange (25) + : +- * Project (24) + : +- * Filter (23) + : +- * ColumnarToRow (22) + : +- Scan parquet default.item (21) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -113,108 +113,108 @@ Join condition: None Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] -(11) Scan parquet default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] +(11) Scan parquet default.customer_address +Output [2]: [ca_address_sk#9, ca_gmt_offset#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] + +(13) Filter [codegen id : 2] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) + +(14) Project [codegen id : 2] +Output [1]: [ca_address_sk#9] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] + +(15) BroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] + +(16) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#9] +Join condition: None + +(17) Project [codegen id : 5] +Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] +Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] + +(18) Scan parquet default.item +Output [2]: [i_item_sk#12, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] +(19) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#12, i_manufact_id#13] -(13) Filter [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) +(20) Filter [codegen id : 4] +Input [2]: [i_item_sk#12, i_manufact_id#13] +Condition : isnotnull(i_item_sk#12) -(14) Scan parquet default.item -Output [2]: [i_category#11, i_manufact_id#10] +(21) Scan parquet default.item +Output [2]: [i_category#14, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_category#11, i_manufact_id#10] +(22) ColumnarToRow [codegen id : 3] +Input [2]: [i_category#14, i_manufact_id#13] -(16) Filter [codegen id : 2] -Input [2]: [i_category#11, i_manufact_id#10] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics)) +(23) Filter [codegen id : 3] +Input [2]: [i_category#14, i_manufact_id#13] +Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) -(17) Project [codegen id : 2] -Output [1]: [i_manufact_id#10 AS i_manufact_id#10#12] -Input [2]: [i_category#11, i_manufact_id#10] +(24) Project [codegen id : 3] +Output [1]: [i_manufact_id#13 AS i_manufact_id#13#15] +Input [2]: [i_category#14, i_manufact_id#13] -(18) BroadcastExchange -Input [1]: [i_manufact_id#10#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(25) BroadcastExchange +Input [1]: [i_manufact_id#13#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact_id#10] -Right keys [1]: [i_manufact_id#10#12] +(26) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_manufact_id#13] +Right keys [1]: [i_manufact_id#13#15] Join condition: None -(20) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] -Join condition: None - -(22) Project [codegen id : 5] -Output [3]: [ss_addr_sk#3, ss_ext_sales_price#4, i_manufact_id#10] -Input [5]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, i_item_sk#9, i_manufact_id#10] - -(23) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_gmt_offset#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] - -(25) Filter [codegen id : 4] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Condition : ((isnotnull(ca_gmt_offset#16) AND (ca_gmt_offset#16 = -5.00)) AND isnotnull(ca_address_sk#15)) - -(26) Project [codegen id : 4] -Output [1]: [ca_address_sk#15] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] - (27) BroadcastExchange -Input [1]: [ca_address_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [2]: [i_item_sk#12, i_manufact_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#12] Join condition: None (29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_manufact_id#10] -Input [4]: [ss_addr_sk#3, ss_ext_sales_price#4, i_manufact_id#10, ca_address_sk#15] +Output [2]: [ss_ext_sales_price#4, i_manufact_id#13] +Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_manufact_id#13] (30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [ss_ext_sales_price#4, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum#18] -Results [2]: [i_manufact_id#10, sum#19] +Results [2]: [i_manufact_id#13, sum#19] (31) Exchange -Input [2]: [i_manufact_id#10, sum#19] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#20] +Input [2]: [i_manufact_id#13, sum#19] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#20] (32) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#10, sum#19] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#19] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] (33) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] @@ -242,47 +242,47 @@ Join condition: None Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] -(39) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#9, i_manufact_id#10] +(39) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#9] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [cs_bill_addr_sk#24] +Right keys [1]: [ca_address_sk#9] Join condition: None (41) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#10] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#9, i_manufact_id#10] +Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] (42) ReusedExchange [Reuses operator id: 27] -Output [1]: [ca_address_sk#15] +Output [2]: [i_item_sk#12, i_manufact_id#13] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#12] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#10] -Input [4]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#10, ca_address_sk#15] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#13] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_manufact_id#13] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum#27] -Results [2]: [i_manufact_id#10, sum#28] +Results [2]: [i_manufact_id#13, sum#28] (46) Exchange -Input [2]: [i_manufact_id#10, sum#28] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#29] +Input [2]: [i_manufact_id#13, sum#28] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#10, sum#28] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#28] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] (48) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] @@ -310,69 +310,69 @@ Join condition: None Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] -(54) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#9, i_manufact_id#10] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#9] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (56) Project [codegen id : 17] -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, i_manufact_id#10] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, i_item_sk#9, i_manufact_id#10] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] (57) ReusedExchange [Reuses operator id: 27] -Output [1]: [ca_address_sk#15] +Output [2]: [i_item_sk#12, i_manufact_id#13] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#12] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#10] -Input [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, i_manufact_id#10, ca_address_sk#15] +Output [2]: [ws_ext_sales_price#35, i_manufact_id#13] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_manufact_id#13] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [ws_ext_sales_price#35, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] Aggregate Attributes [1]: [sum#36] -Results [2]: [i_manufact_id#10, sum#37] +Results [2]: [i_manufact_id#13, sum#37] (61) Exchange -Input [2]: [i_manufact_id#10, sum#37] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#38] +Input [2]: [i_manufact_id#13, sum#37] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#38] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#10, sum#37] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#37] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] (63) Union (64) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#10, total_sales#22] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, total_sales#22] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(total_sales#22)] Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] +Results [3]: [i_manufact_id#13, sum#43, isEmpty#44] (65) Exchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#45] +Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#45] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#10] +Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(total_sales#22)] Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_manufact_id#10, sum(total_sales#22)#46 AS total_sales#47] +Results [2]: [i_manufact_id#13, sum(total_sales#22)#46 AS total_sales#47] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#10, total_sales#47] +Input [2]: [i_manufact_id#13, total_sales#47] +Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] 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 410def2466e1a..14787f0bbce7b 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 @@ -14,9 +14,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] @@ -33,28 +33,28 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 + BroadcastExchange #5 WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (12) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter @@ -62,9 +62,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (11) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] @@ -74,9 +74,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 + ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter @@ -84,9 +84,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] @@ -96,6 +96,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 + ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt index d7a8c103285cb..6492918d3aa13 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt @@ -6,117 +6,117 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) + : +- * BroadcastHashJoin Inner BuildLeft (9) + : :- BroadcastExchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.date_dim (1) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_sales (6) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.date_dim (11) + +- Scan parquet default.item (11) -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) -(4) Scan parquet default.item -Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct +(4) Project [codegen id : 1] +Output [2]: [d_date_sk#1, d_year#2] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(5) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(5) BroadcastExchange +Input [2]: [d_date_sk#1, d_year#2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(6) Filter [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 1)) AND isnotnull(i_item_sk#4)) +(6) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(7) ColumnarToRow +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(8) BroadcastExchange -Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(8) Filter +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] +Output [3]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7] +Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +(11) Scan parquet default.item +Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (13) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 2000)) AND isnotnull(d_date_sk#9)) +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) (14) Project [codegen id : 2] -Output [2]: [d_date_sk#9, d_year#10] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (15) BroadcastExchange -Input [2]: [d_date_sk#9, d_year#10] +Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#8] Join condition: None (17) Project [codegen id : 3] -Output [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9, d_year#10] +Output [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Input [6]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] (18) HashAggregate [codegen id : 3] -Input [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Input [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] (19) Exchange -Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] -Arguments: hashpartitioning(d_year#10, i_brand#6, i_brand_id#5, 5), true, [id=#15] +Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] -Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [4]: [d_year#10, i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] +Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] +Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject -Input [4]: [d_year#10, brand_id#17, brand#18, ext_price#19] -Arguments: 100, [d_year#10 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#10, brand_id#17, brand#18, ext_price#19] +Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt index 8ed500d84390c..f4aaf3df75135 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt index a1257cd292e48..b8d8aa358d532 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt @@ -6,115 +6,115 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) + : +- * BroadcastHashJoin Inner BuildLeft (9) + : :- BroadcastExchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.date_dim (1) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_sales (6) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.date_dim (11) + +- Scan parquet default.item (11) -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) -(4) Scan parquet default.item -Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct +(4) Project [codegen id : 1] +Output [1]: [d_date_sk#1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(5) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(5) BroadcastExchange +Input [1]: [d_date_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(6) Filter [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 28)) AND isnotnull(i_item_sk#4)) +(6) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(7) ColumnarToRow +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(8) BroadcastExchange -Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(8) Filter +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] +Output [2]: [ss_item_sk#6, ss_ext_sales_price#7] +Input [4]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +(11) Scan parquet default.item +Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (13) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 1999)) AND isnotnull(d_date_sk#9)) +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 28)) AND isnotnull(i_item_sk#8)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (15) BroadcastExchange -Input [1]: [d_date_sk#9] +Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#8] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [5]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9] +Output [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Input [5]: [ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] (18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Keys [2]: [i_brand#6, i_brand_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Input [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Keys [2]: [i_brand#10, i_brand_id#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#6, i_brand_id#5, sum#14] +Results [3]: [i_brand#10, i_brand_id#9, sum#14] (19) Exchange -Input [3]: [i_brand#6, i_brand_id#5, sum#14] -Arguments: hashpartitioning(i_brand#6, i_brand_id#5, 5), true, [id=#15] +Input [3]: [i_brand#10, i_brand_id#9, sum#14] +Arguments: hashpartitioning(i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#6, i_brand_id#5, sum#14] -Keys [2]: [i_brand#6, i_brand_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [3]: [i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] +Input [3]: [i_brand#10, i_brand_id#9, sum#14] +Keys [2]: [i_brand#10, i_brand_id#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] +Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt index b0d0e0d809441..4f375c80678e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 3f8106c96379a..3007b11a1a860 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- BroadcastExchange (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildLeft (41) - : : : :- BroadcastExchange (37) - : : : : +- * Project (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.date_dim (38) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Project (7) + : : : : : : +- * Filter (6) + : : : : : : +- * ColumnarToRow (5) + : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.customer_demographics (11) + : : : : +- BroadcastExchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.date_dim (18) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.item (26) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.inventory (48) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildLeft (49) + : : : :- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildLeft (43) + : : : : :- BroadcastExchange (39) + : : : : : +- * Project (38) + : : : : : +- * Filter (37) + : : : : : +- * ColumnarToRow (36) + : : : : : +- Scan parquet default.date_dim (35) + : : : : +- * Filter (42) + : : : : +- * ColumnarToRow (41) + : : : : +- Scan parquet default.date_dim (40) + : : : +- * Filter (48) + : : : +- * ColumnarToRow (47) + : : : +- Scan parquet default.inventory (46) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -185,7 +185,7 @@ Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, c (24) Exchange Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] +Arguments: hashpartitioning(cs_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] @@ -207,101 +207,101 @@ Condition : isnotnull(i_item_sk#19) (29) Exchange Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), true, [id=#21] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 10] +(31) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 10] +(32) Project [codegen id : 8] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Scan parquet default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(33) Exchange +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] + +(34) Sort [codegen id : 9] +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 + +(35) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(35) Filter [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(37) Filter [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(36) Project [codegen id : 8] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(38) Project [codegen id : 10] +Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] +(39) BroadcastExchange +Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] -(38) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#28] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow -Input [2]: [d_date_sk#27, d_week_seq#28] +(41) ColumnarToRow +Input [2]: [d_date_sk#28, d_week_seq#29] -(40) Filter -Input [2]: [d_date_sk#27, d_week_seq#28] -Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) +(42) Filter +Input [2]: [d_date_sk#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#24] -Right keys [1]: [d_week_seq#28] +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [d_week_seq#25] +Right keys [1]: [d_week_seq#29] Join condition: None -(42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] - -(43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] -Join condition: (d_date#16 > d_date#23 + 5 days) - -(45) Project [codegen id : 10] -Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +(44) Project [codegen id : 11] +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] -(46) Exchange -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] +(45) BroadcastExchange +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] -(47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.inventory +(46) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 13] +(47) ColumnarToRow Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(50) Filter [codegen id : 13] +(48) Filter Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_date_sk#28] +Right keys [1]: [inv_date_sk#31] +Join condition: None + +(50) Project [codegen id : 13] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] + (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, d_date_sk#27] -Right keys [2]: [inv_item_sk#32, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) +Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] +Right keys [2]: [inv_item_sk#32, d_date_sk#23] +Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -397,7 +397,7 @@ Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] -Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), true, [id=#44] +Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), ENSURE_REQUIREMENTS, [id=#44] (73) Sort [codegen id : 19] Input [2]: [cr_item_sk#42, cr_order_number#43] @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index 918508787c4b0..b88505ad7b9bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] + WholeStageCodegen (9) + Sort [cs_item_sk,cs_sold_date_sk] InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + Exchange [cs_item_sk,cs_sold_date_sk] #3 + WholeStageCodegen (8) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] + Sort [inv_item_sk,d_date_sk] InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 + Exchange [inv_item_sk,d_date_sk] #9 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [d_date_sk,inv_date_sk] InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastExchange #10 + WholeStageCodegen (11) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 6e757528a3e68..6813696266ac5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -1,343 +1,343 @@ == Physical Plan == TakeOrderedAndProject (61) +- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (38) - : +- * SortMergeJoin Inner (37) - : :- * Sort (11) - : : +- Exchange (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.customer (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.customer_address (4) - : +- * Sort (36) - : +- Exchange (35) - : +- * Filter (34) - : +- * HashAggregate (33) - : +- Exchange (32) - : +- * HashAggregate (31) - : +- * Project (30) - : +- * SortMergeJoin Inner (29) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.catalog_returns (12) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- * Sort (28) - : +- Exchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- BroadcastExchange (58) - +- * Filter (57) - +- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Filter (41) - : : +- * ColumnarToRow (40) - : : +- Scan parquet default.catalog_returns (39) - : +- ReusedExchange (42) - +- * Sort (48) - +- ReusedExchange (47) - - -(1) Scan parquet default.customer -Output [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] + +- * SortMergeJoin Inner (59) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_returns (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer_address (13) + : +- BroadcastExchange (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * SortMergeJoin Inner (34) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (26) + : : : +- * ColumnarToRow (25) + : : : +- Scan parquet default.catalog_returns (24) + : : +- ReusedExchange (27) + : +- * Sort (33) + : +- ReusedExchange (32) + +- * Sort (58) + +- Exchange (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Filter (50) + : +- * ColumnarToRow (49) + : +- Scan parquet default.customer (48) + +- BroadcastExchange (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.customer_address (51) + + +(1) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] (3) Filter [codegen id : 2] -Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Condition : ((isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) AND isnotnull(cr_returning_customer_sk#2)) -(4) Scan parquet default.customer_address -Output [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#5, d_year#6] 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 +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Input [2]: [d_date_sk#5, d_year#6] (6) Filter [codegen id : 1] -Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Condition : ((isnotnull(ca_state#14) AND (ca_state#14 = GA)) AND isnotnull(ca_address_sk#7)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) + +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_year#6] -(7) BroadcastExchange -Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] +(8) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#7] +(9) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] Join condition: None -(9) Project [codegen id : 2] -Output [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Input [18]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6, ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +(10) Project [codegen id : 2] +Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] -(10) Exchange -Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Arguments: hashpartitioning(c_customer_sk#1, 5), true, [id=#20] +(11) Exchange +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#8] -(11) Sort [codegen id : 3] -Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 +(12) Sort [codegen id : 3] +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(13) Scan parquet default.customer_address +Output [2]: [ca_address_sk#9, ca_state#10] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 5] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct -(14) Filter [codegen id : 5] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Condition : ((isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) AND isnotnull(cr_returning_customer_sk#22)) +(14) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#9, ca_state#10] -(15) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +(15) Filter [codegen id : 4] +Input [2]: [ca_address_sk#9, ca_state#10] +Condition : (isnotnull(ca_address_sk#9) AND isnotnull(ca_state#10)) -(16) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#25, d_year#26] +(16) Exchange +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: hashpartitioning(ca_address_sk#9, 5), ENSURE_REQUIREMENTS, [id=#11] -(17) Filter [codegen id : 4] -Input [2]: [d_date_sk#25, d_year#26] -Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2000)) AND isnotnull(d_date_sk#25)) +(17) Sort [codegen id : 5] +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 -(18) Project [codegen id : 4] -Output [1]: [d_date_sk#25] -Input [2]: [d_date_sk#25, d_year#26] +(18) SortMergeJoin [codegen id : 6] +Left keys [1]: [cr_returning_addr_sk#3] +Right keys [1]: [ca_address_sk#9] +Join condition: None -(19) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +(19) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] + +(20) HashAggregate [codegen id : 6] +Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum#12] +Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] + +(21) Exchange +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] +Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] + +(22) HashAggregate [codegen id : 15] +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#15] +Results [3]: [cr_returning_customer_sk#2 AS ctr_customer_sk#16, ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#15,17,2) AS ctr_total_return#18] + +(23) Filter [codegen id : 15] +Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] +Condition : isnotnull(ctr_total_return#18) + +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#25] -Join condition: None +(25) ColumnarToRow [codegen id : 8] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -(21) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] +(26) Filter [codegen id : 8] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Condition : (isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) -(22) Exchange -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), true, [id=#28] +(27) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#5] -(23) Sort [codegen id : 6] -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] +Join condition: None -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#7, ca_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct +(29) Project [codegen id : 8] +Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] -(25) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#7, ca_state#14] +(30) Exchange +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#19] -(26) Filter [codegen id : 7] -Input [2]: [ca_address_sk#7, ca_state#14] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#14)) +(31) Sort [codegen id : 9] +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 -(27) Exchange -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: hashpartitioning(ca_address_sk#7, 5), true, [id=#29] +(32) ReusedExchange [Reuses operator id: 16] +Output [2]: [ca_address_sk#9, ca_state#10] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 +(33) Sort [codegen id : 11] +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [cr_returning_addr_sk#23] -Right keys [1]: [ca_address_sk#7] +(34) SortMergeJoin [codegen id : 12] +Left keys [1]: [cr_returning_addr_sk#3] +Right keys [1]: [ca_address_sk#9] Join condition: None -(30) Project [codegen id : 9] -Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] - -(31) HashAggregate [codegen id : 9] -Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] - -(32) Exchange -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] -Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), true, [id=#32] - -(33) HashAggregate [codegen id : 10] -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#33] -Results [3]: [cr_returning_customer_sk#22 AS ctr_customer_sk#34, ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#33,17,2) AS ctr_total_return#36] - -(34) Filter [codegen id : 10] -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Condition : isnotnull(ctr_total_return#36) - -(35) Exchange -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Arguments: hashpartitioning(ctr_customer_sk#34, 5), true, [id=#37] - -(36) Sort [codegen id : 11] -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Arguments: [ctr_customer_sk#34 ASC NULLS FIRST], false, 0 - -(37) SortMergeJoin [codegen id : 20] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ctr_customer_sk#34] -Join condition: None +(35) Project [codegen id : 12] +Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] + +(36) HashAggregate [codegen id : 12] +Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] + +(37) Exchange +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] +Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#22] + +(38) HashAggregate [codegen id : 13] +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#23] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#23,17,2) AS ctr_total_return#18] + +(39) HashAggregate [codegen id : 13] +Input [2]: [ctr_state#17, ctr_total_return#18] +Keys [1]: [ctr_state#17] +Functions [1]: [partial_avg(ctr_total_return#18)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ctr_state#17, sum#26, count#27] + +(40) Exchange +Input [3]: [ctr_state#17, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#28] + +(41) HashAggregate [codegen id : 14] +Input [3]: [ctr_state#17, sum#26, count#27] +Keys [1]: [ctr_state#17] +Functions [1]: [avg(ctr_total_return#18)] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#29] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#29) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17 AS ctr_state#17#31] + +(42) Filter [codegen id : 14] +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] +Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) + +(43) BroadcastExchange +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#32] + +(44) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ctr_state#17] +Right keys [1]: [ctr_state#17#31] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) + +(45) Project [codegen id : 15] +Output [2]: [ctr_customer_sk#16, ctr_total_return#18] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] + +(46) Exchange +Input [2]: [ctr_customer_sk#16, ctr_total_return#18] +Arguments: hashpartitioning(ctr_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#33] + +(47) Sort [codegen id : 16] +Input [2]: [ctr_customer_sk#16, ctr_total_return#18] +Arguments: [ctr_customer_sk#16 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.customer +Output [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct + +(49) ColumnarToRow [codegen id : 18] +Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -(38) Project [codegen id : 20] -Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36] -Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +(50) Filter [codegen id : 18] +Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) -(39) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(51) Scan parquet default.customer_address +Output [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct -(40) ColumnarToRow [codegen id : 13] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(52) ColumnarToRow [codegen id : 17] +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -(41) Filter [codegen id : 13] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Condition : (isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) +(53) Filter [codegen id : 17] +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) -(42) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#25] +(54) BroadcastExchange +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#25] +(55) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#9] Join condition: None -(44) Project [codegen id : 13] -Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] +(56) Project [codegen id : 18] +Output [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Input [18]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -(45) Exchange -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), true, [id=#38] +(57) Exchange +Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: hashpartitioning(c_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#51] -(46) Sort [codegen id : 14] -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 19] +Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 -(47) ReusedExchange [Reuses operator id: 27] -Output [2]: [ca_address_sk#7, ca_state#14] - -(48) Sort [codegen id : 16] -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 - -(49) SortMergeJoin [codegen id : 17] -Left keys [1]: [cr_returning_addr_sk#23] -Right keys [1]: [ca_address_sk#7] +(59) SortMergeJoin [codegen id : 20] +Left keys [1]: [ctr_customer_sk#16] +Right keys [1]: [c_customer_sk#34] Join condition: None -(50) Project [codegen id : 17] -Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] - -(51) HashAggregate [codegen id : 17] -Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum#39] -Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] - -(52) Exchange -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] -Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), true, [id=#41] - -(53) HashAggregate [codegen id : 18] -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#42] -Results [2]: [ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#42,17,2) AS ctr_total_return#36] - -(54) HashAggregate [codegen id : 18] -Input [2]: [ctr_state#35, ctr_total_return#36] -Keys [1]: [ctr_state#35] -Functions [1]: [partial_avg(ctr_total_return#36)] -Aggregate Attributes [2]: [sum#43, count#44] -Results [3]: [ctr_state#35, sum#45, count#46] - -(55) Exchange -Input [3]: [ctr_state#35, sum#45, count#46] -Arguments: hashpartitioning(ctr_state#35, 5), true, [id=#47] - -(56) HashAggregate [codegen id : 19] -Input [3]: [ctr_state#35, sum#45, count#46] -Keys [1]: [ctr_state#35] -Functions [1]: [avg(ctr_total_return#36)] -Aggregate Attributes [1]: [avg(ctr_total_return#36)#48] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#36)#48) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35 AS ctr_state#35#50] - -(57) Filter [codegen id : 19] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) - -(58) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#51] - -(59) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ctr_state#35] -Right keys [1]: [ctr_state#35#50] -Join condition: (cast(ctr_total_return#36 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) - (60) Project [codegen id : 20] -Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] -Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Output [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Input [18]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] -Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#36 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] +Input [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, ca_street_number#40 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#42 ASC NULLS FIRST, ca_suite_number#43 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#46 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#49 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index c603ab5194286..99677b6e39736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,48 +1,29 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] WholeStageCodegen (20) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] - SortMergeJoin [c_customer_sk,ctr_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #1 - WholeStageCodegen (2) - Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - InputAdapter - WholeStageCodegen (11) - Sort [ctr_customer_sk] - InputAdapter - Exchange [ctr_customer_sk] #3 - WholeStageCodegen (10) + SortMergeJoin [ctr_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (16) + Sort [ctr_customer_sk] + InputAdapter + Exchange [ctr_customer_sk] #1 + WholeStageCodegen (15) + Project [ctr_customer_sk,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #4 - WholeStageCodegen (9) + Exchange [cr_returning_customer_sk,ca_state] #2 + WholeStageCodegen (6) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] SortMergeJoin [cr_returning_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [cr_returning_addr_sk] InputAdapter - Exchange [cr_returning_addr_sk] #5 - WholeStageCodegen (5) + Exchange [cr_returning_addr_sk] #3 + WholeStageCodegen (2) Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] @@ -50,55 +31,74 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (8) + WholeStageCodegen (5) Sort [ca_address_sk] InputAdapter - Exchange [ca_address_sk] #7 - WholeStageCodegen (7) + Exchange [ca_address_sk] #5 + WholeStageCodegen (4) Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] + InputAdapter + Exchange [ctr_state] #7 + WholeStageCodegen (13) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #8 + WholeStageCodegen (12) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + SortMergeJoin [cr_returning_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (9) + Sort [cr_returning_addr_sk] + InputAdapter + Exchange [cr_returning_addr_sk] #9 + WholeStageCodegen (8) + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_returned_date_sk,cr_returning_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (11) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #5 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (19) - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #9 - WholeStageCodegen (18) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + WholeStageCodegen (19) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #10 + WholeStageCodegen (18) + Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #10 - WholeStageCodegen (17) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - SortMergeJoin [cr_returning_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (14) - Sort [cr_returning_addr_sk] - InputAdapter - Exchange [cr_returning_addr_sk] #11 - WholeStageCodegen (13) - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (16) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #7 + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (17) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 4e85516b594f7..6bcbe470cec50 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -8,206 +8,206 @@ +- * BroadcastHashJoin Inner BuildRight (41) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.customer_demographics (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.household_demographics (10) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.customer_address (17) + : :- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildLeft (15) + : : : : :- BroadcastExchange (11) + : : : : : +- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_returns (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.date_dim (4) + : : : : +- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet default.customer (12) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.household_demographics (17) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.customer_address (24) : +- BroadcastExchange (34) - : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.catalog_returns (24) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.date_dim (27) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_demographics (31) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(1) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +(1) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +(2) ColumnarToRow [codegen id : 2] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -(3) Filter [codegen id : 7] -Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] -Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#4)) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) +(3) Filter [codegen id : 2] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Condition : ((isnotnull(cr_call_center_sk#3) AND isnotnull(cr_returned_date_sk#1)) AND isnotnull(cr_returning_customer_sk#2)) -(4) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +(4) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_moy#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] (6) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -Condition : ((((cd_marital_status#6 = M) AND (cd_education_status#7 = Unknown)) OR ((cd_marital_status#6 = W) AND (cd_education_status#7 = Advanced Degree))) AND isnotnull(cd_demo_sk#5)) +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 11)) AND isnotnull(d_date_sk#5)) + +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#5] +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -(7) BroadcastExchange -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] +(8) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#5] +(9) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] Join condition: None -(9) Project [codegen id : 7] -Output [5]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] -Input [7]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +(10) Project [codegen id : 2] +Output [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, d_date_sk#5] -(10) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#9, hd_buy_potential#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] -ReadSchema: struct +(11) BroadcastExchange +Input [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] -(11) ColumnarToRow [codegen id : 2] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] - -(12) Filter [codegen id : 2] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND StartsWith(hd_buy_potential#10, Unknown)) AND isnotnull(hd_demo_sk#9)) +(12) Scan parquet default.customer +Output [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct -(13) Project [codegen id : 2] -Output [1]: [hd_demo_sk#9] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +(13) ColumnarToRow +Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -(14) BroadcastExchange -Input [1]: [hd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +(14) Filter +Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Condition : (((isnotnull(c_customer_sk#10) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_current_cdemo_sk#11)) AND isnotnull(c_current_hdemo_sk#12)) (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#9] +Left keys [1]: [cr_returning_customer_sk#2] +Right keys [1]: [c_customer_sk#10] Join condition: None (16) Project [codegen id : 7] -Output [4]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] -Input [6]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, hd_demo_sk#9] +Output [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Input [7]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -(17) Scan parquet default.customer_address -Output [2]: [ca_address_sk#12, ca_gmt_offset#13] +(17) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#14, hd_buy_potential#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] (19) Filter [codegen id : 3] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] -Condition : ((isnotnull(ca_gmt_offset#13) AND (ca_gmt_offset#13 = -7.00)) AND isnotnull(ca_address_sk#12)) +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Condition : ((isnotnull(hd_buy_potential#15) AND StartsWith(hd_buy_potential#15, Unknown)) AND isnotnull(hd_demo_sk#14)) (20) Project [codegen id : 3] -Output [1]: [ca_address_sk#12] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Output [1]: [hd_demo_sk#14] +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] (21) BroadcastExchange -Input [1]: [ca_address_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [hd_demo_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#4] -Right keys [1]: [ca_address_sk#12] +Left keys [1]: [c_current_hdemo_sk#12] +Right keys [1]: [hd_demo_sk#14] Join condition: None (23) Project [codegen id : 7] -Output [3]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7] -Input [5]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, ca_address_sk#12] +Output [4]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13] +Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13, hd_demo_sk#14] -(24) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#17, ca_gmt_offset#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 5] -Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +(25) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -(26) Filter [codegen id : 5] -Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Condition : ((isnotnull(cr_call_center_sk#17) AND isnotnull(cr_returned_date_sk#15)) AND isnotnull(cr_returning_customer_sk#16)) +(26) Filter [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -7.00)) AND isnotnull(ca_address_sk#17)) -(27) Scan parquet default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct +(27) Project [codegen id : 4] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -(28) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +(28) BroadcastExchange +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) Filter [codegen id : 4] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 1998)) AND (d_moy#21 = 11)) AND isnotnull(d_date_sk#19)) +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#13] +Right keys [1]: [ca_address_sk#17] +Join condition: None -(30) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +(30) Project [codegen id : 7] +Output [3]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11] +Input [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13, ca_address_sk#17] -(31) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +(31) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +ReadSchema: struct -(32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cr_returned_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join condition: None +(32) ColumnarToRow [codegen id : 5] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -(33) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Input [5]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18, d_date_sk#19] +(33) Filter [codegen id : 5] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Condition : ((((cd_marital_status#21 = M) AND (cd_education_status#22 = Unknown)) OR ((cd_marital_status#21 = W) AND (cd_education_status#22 = Advanced Degree))) AND isnotnull(cd_demo_sk#20)) (34) BroadcastExchange -Input [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cr_returning_customer_sk#16] +Left keys [1]: [c_current_cdemo_sk#11] +Right keys [1]: [cd_demo_sk#20] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18] -Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Output [4]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] @@ -228,35 +228,35 @@ Input [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#17] +Left keys [1]: [cr_call_center_sk#3] Right keys [1]: [cc_call_center_sk#24] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] -Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] +Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Input [8]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#18))] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#4))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] (44) Exchange -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, 5), true, [id=#31] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] -Functions [1]: [sum(UnscaledValue(cr_net_loss#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#18))#32] -Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#18))#32,17,2) AS Returns_Loss#36] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] +Functions [1]: [sum(UnscaledValue(cr_net_loss#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#4))#32] +Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#4))#32,17,2) AS Returns_Loss#36] (46) Exchange Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] -Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), true, [id=#37] +Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#37] (47) Sort [codegen id : 9] Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index 87beb3b565cc1..6c8d629feed3e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -10,58 +10,58 @@ WholeStageCodegen (9) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] - Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [c_customer_sk,cd_marital_status,cd_education_status] + Project [cr_call_center_sk,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_customer_sk,c_current_hdemo_sk,c_current_addr_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #6 + WholeStageCodegen (4) Project [ca_address_sk] Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 + BroadcastExchange #7 WholeStageCodegen (5) - Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + Filter [cd_marital_status,cd_education_status,cd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) 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 2d76deefcaa36..f6c5258701525 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 @@ -34,24 +34,24 @@ TakeOrderedAndProject (160) : +- * Sort (46) : +- Exchange (45) : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet default.customer (26) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * ColumnarToRow (31) - : : +- Scan parquet default.customer_address (30) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_demographics (38) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (38) + : : +- * SortMergeJoin Inner (37) + : : :- * Sort (31) + : : : +- Exchange (30) + : : : +- * Project (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.customer (26) + : : +- * Sort (36) + : : +- Exchange (35) + : : +- * Filter (34) + : : +- * ColumnarToRow (33) + : : +- Scan parquet default.customer_demographics (32) + : +- BroadcastExchange (42) + : +- * Filter (41) + : +- * ColumnarToRow (40) + : +- Scan parquet default.customer_address (39) :- * HashAggregate (76) : +- Exchange (75) : +- * HashAggregate (74) @@ -266,7 +266,7 @@ Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6 (24) Exchange Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] (25) Sort [codegen id : 5] Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -279,89 +279,89 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(27) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) -(29) Project [codegen id : 7] +(29) Project [codegen id : 6] Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(30) Scan parquet default.customer_address -Output [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(32) Filter [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) - -(33) BroadcastExchange -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] -Join condition: None - -(35) Project [codegen id : 7] -Output [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [8]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(36) Exchange -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#32] +(30) Exchange +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(37) Sort [codegen id : 8] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +(31) Sort [codegen id : 7] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#33] +(32) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 9] -Input [1]: [cd_demo_sk#33] +(33) ColumnarToRow [codegen id : 8] +Input [1]: [cd_demo_sk#28] -(40) Filter [codegen id : 9] -Input [1]: [cd_demo_sk#33] -Condition : isnotnull(cd_demo_sk#33) +(34) Filter [codegen id : 8] +Input [1]: [cd_demo_sk#28] +Condition : isnotnull(cd_demo_sk#28) -(41) Exchange -Input [1]: [cd_demo_sk#33] -Arguments: hashpartitioning(cd_demo_sk#33, 5), true, [id=#34] +(35) Exchange +Input [1]: [cd_demo_sk#28] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(42) Sort [codegen id : 10] -Input [1]: [cd_demo_sk#33] -Arguments: [cd_demo_sk#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 9] +Input [1]: [cd_demo_sk#28] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 11] +(37) SortMergeJoin [codegen id : 11] Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#33] +Right keys [1]: [cd_demo_sk#28] +Join condition: None + +(38) Project [codegen id : 11] +Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] + +(39) Scan parquet default.customer_address +Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(40) ColumnarToRow [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] + +(41) Filter [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) + +(42) BroadcastExchange +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#24] +Right keys [1]: [ca_address_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30, cd_demo_sk#33] +Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#35] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -370,26 +370,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (48) Project [codegen id : 13] -Output [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Output [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] (49) HashAggregate [codegen id : 13] -Input [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [4]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28] +Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Results [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (50) Exchange -Input [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, ca_state#29, ca_county#28, 5), true, [id=#71] +Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 5), ENSURE_REQUIREMENTS, [id=#71] (51) HashAggregate [codegen id : 14] -Input [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [4]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28] +Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#72, avg(agg2#37)#73, avg(agg3#38)#74, avg(agg4#39)#75, avg(agg5#40)#76, avg(agg6#41)#77, avg(agg7#42)#78] -Results [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] +Results [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] (52) ReusedExchange [Reuses operator id: 24] Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -417,41 +417,41 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (61) BroadcastExchange -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (62) BroadcastHashJoin [codegen id : 21] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_state#29, ca_country#30] +Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_state#32, ca_country#33] (64) Exchange -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#87] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#87] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 41] +(66) ReusedExchange [Reuses operator id: 35] Output [1]: [cd_demo_sk#88] (67) Sort [codegen id : 24] @@ -464,15 +464,15 @@ Right keys [1]: [cd_demo_sk#88] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30, cd_demo_sk#88] +Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33, cd_demo_sk#88] (70) Exchange -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#89] +Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#89] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] +Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] @@ -481,26 +481,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#19, ca_country#30, ca_state#29, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] +Output [10]: [i_item_id#19, ca_country#33, ca_state#32, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#19, ca_country#30, ca_state#29, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [3]: [i_item_id#19, ca_country#30, ca_state#29] +Input [10]: [i_item_id#19, ca_country#33, ca_state#32, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103] -Results [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Results [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] (75) Exchange -Input [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, ca_state#29, 5), true, [id=#118] +Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, 5), ENSURE_REQUIREMENTS, [id=#118] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Keys [3]: [i_item_id#19, ca_country#30, ca_state#29] +Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#119, avg(agg2#37)#120, avg(agg3#38)#121, avg(agg4#39)#122, avg(agg5#40)#123, avg(agg6#41)#124, avg(agg7#42)#125] -Results [11]: [i_item_id#19, ca_country#30, ca_state#29, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] +Results [11]: [i_item_id#19, ca_country#33, ca_state#32, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] (77) ReusedExchange [Reuses operator id: 24] Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -528,45 +528,45 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#27, ca_country#30] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [2]: [ca_address_sk#30, ca_country#33] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (87) BroadcastExchange -Input [2]: [ca_address_sk#27, ca_country#30] +Input [2]: [ca_address_sk#30, ca_country#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#134] (88) BroadcastHashJoin [codegen id : 35] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_country#30] +Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] +Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_country#33] (90) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#135] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#135] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 41] +(92) ReusedExchange [Reuses operator id: 35] Output [1]: [cd_demo_sk#136] (93) Sort [codegen id : 38] @@ -579,15 +579,15 @@ Right keys [1]: [cd_demo_sk#136] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30, cd_demo_sk#136] +Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33, cd_demo_sk#136] (96) Exchange -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#137] +Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#137] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] +Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] @@ -596,26 +596,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#19, ca_country#30, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#30] +Output [9]: [i_item_id#19, ca_country#33, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#33] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#19, ca_country#30, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [2]: [i_item_id#19, ca_country#30] +Input [9]: [i_item_id#19, ca_country#33, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [2]: [i_item_id#19, ca_country#33] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149, sum#150, count#151] -Results [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Results [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] (101) Exchange -Input [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, 5), true, [id=#166] +Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, 5), ENSURE_REQUIREMENTS, [id=#166] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Keys [2]: [i_item_id#19, ca_country#30] +Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Keys [2]: [i_item_id#19, ca_country#33] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#167, avg(agg2#37)#168, avg(agg3#38)#169, avg(agg4#39)#170, avg(agg5#40)#171, avg(agg6#41)#172, avg(agg7#42)#173] -Results [11]: [i_item_id#19, ca_country#30, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] +Results [11]: [i_item_id#19, ca_country#33, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] (103) Scan parquet default.catalog_sales Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] @@ -674,35 +674,35 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#27, ca_state#29] +Output [2]: [ca_address_sk#30, ca_state#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#27, ca_state#29] +Input [2]: [ca_address_sk#30, ca_state#32] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#27, ca_state#29] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [2]: [ca_address_sk#30, ca_state#32] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#27] -Input [2]: [ca_address_sk#27, ca_state#29] +Output [1]: [ca_address_sk#30] +Input [2]: [ca_address_sk#30, ca_state#32] (120) BroadcastExchange -Input [1]: [ca_address_sk#27] +Input [1]: [ca_address_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#183] (121) BroadcastHashJoin [codegen id : 46] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (122) Project [codegen id : 46] Output [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30] (123) BroadcastExchange Input [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] @@ -765,7 +765,7 @@ Results [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, co (136) Exchange Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Arguments: hashpartitioning(i_item_id#19, 5), true, [id=#215] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#215] (137) HashAggregate [codegen id : 50] Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] @@ -860,7 +860,7 @@ Results [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#2 (157) Exchange Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] -Arguments: SinglePartition, true, [id=#262] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#262] (158) HashAggregate [codegen id : 58] Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] @@ -872,6 +872,6 @@ Results [11]: [null AS i_item_id#270, null AS ca_country#271, null AS ca_state#2 (159) Union (160) TakeOrderedAndProject -Input [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] -Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#29 ASC NULLS FIRST, ca_county#28 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Arguments: 100, [ca_country#33 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] 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 5514e335f1b51..4566929712713 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 @@ -54,37 +54,37 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag 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] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,c_birth_year] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (6) 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 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 default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (9) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #8 + WholeStageCodegen (8) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] 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 @@ -130,7 +130,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (24) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #9 + ReusedExchange [cd_demo_sk] #8 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 @@ -177,7 +177,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (38) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #9 + ReusedExchange [cd_demo_sk] #8 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 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index a7f328537b7ac..04ff822b1ce52 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- BroadcastExchange (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildLeft (41) - : : : :- BroadcastExchange (37) - : : : : +- * Project (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.date_dim (38) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Project (7) + : : : : : : +- * Filter (6) + : : : : : : +- * ColumnarToRow (5) + : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.customer_demographics (11) + : : : : +- BroadcastExchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.date_dim (18) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.item (26) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.inventory (48) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildLeft (49) + : : : :- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildLeft (43) + : : : : :- BroadcastExchange (39) + : : : : : +- * Project (38) + : : : : : +- * Filter (37) + : : : : : +- * ColumnarToRow (36) + : : : : : +- Scan parquet default.date_dim (35) + : : : : +- * Filter (42) + : : : : +- * ColumnarToRow (41) + : : : : +- Scan parquet default.date_dim (40) + : : : +- * Filter (48) + : : : +- * ColumnarToRow (47) + : : : +- Scan parquet default.inventory (46) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -185,7 +185,7 @@ Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, c (24) Exchange Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] +Arguments: hashpartitioning(cs_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] @@ -207,101 +207,101 @@ Condition : isnotnull(i_item_sk#19) (29) Exchange Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), true, [id=#21] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 10] +(31) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 10] +(32) Project [codegen id : 8] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Scan parquet default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(33) Exchange +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] + +(34) Sort [codegen id : 9] +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 + +(35) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(35) Filter [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(37) Filter [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(36) Project [codegen id : 8] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(38) Project [codegen id : 10] +Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] +(39) BroadcastExchange +Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] -(38) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#28] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow -Input [2]: [d_date_sk#27, d_week_seq#28] +(41) ColumnarToRow +Input [2]: [d_date_sk#28, d_week_seq#29] -(40) Filter -Input [2]: [d_date_sk#27, d_week_seq#28] -Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) +(42) Filter +Input [2]: [d_date_sk#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#24] -Right keys [1]: [d_week_seq#28] +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [d_week_seq#25] +Right keys [1]: [d_week_seq#29] Join condition: None -(42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] - -(43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] -Join condition: (d_date#16 > d_date#23 + 5 days) - -(45) Project [codegen id : 10] -Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +(44) Project [codegen id : 11] +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] -(46) Exchange -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] +(45) BroadcastExchange +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] -(47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.inventory +(46) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 13] +(47) ColumnarToRow Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(50) Filter [codegen id : 13] +(48) Filter Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_date_sk#28] +Right keys [1]: [inv_date_sk#31] +Join condition: None + +(50) Project [codegen id : 13] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] + (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, d_date_sk#27] -Right keys [2]: [inv_item_sk#32, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) +Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] +Right keys [2]: [inv_item_sk#32, d_date_sk#23] +Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -397,7 +397,7 @@ Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] -Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), true, [id=#44] +Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), ENSURE_REQUIREMENTS, [id=#44] (73) Sort [codegen id : 19] Input [2]: [cr_item_sk#42, cr_order_number#43] @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index 918508787c4b0..b88505ad7b9bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] + WholeStageCodegen (9) + Sort [cs_item_sk,cs_sold_date_sk] InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + Exchange [cs_item_sk,cs_sold_date_sk] #3 + WholeStageCodegen (8) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] + Sort [inv_item_sk,d_date_sk] InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 + Exchange [inv_item_sk,d_date_sk] #9 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [d_date_sk,inv_date_sk] InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastExchange #10 + WholeStageCodegen (11) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12)