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 c41686da79487..828f768f17701 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 @@ -348,12 +348,30 @@ object JoinReorderDP extends PredicateHelper with Logging { } } + /** + * To identify the plan with smaller computational cost, + * we use the weighted geometric mean of ratio of rows and the ratio of sizes in bytes. + * + * There are other ways to combine these values as a cost comparison function. + * Some of these, that we have experimented with, but have gotten worse result, + * than with the current one: + * 1) Weighted arithmetic mean of these two ratios - adding up fractions puts + * less emphasis on ratios between 0 and 1. Ratios 10 and 0.1 should be considered + * to be just as strong evidences in opposite directions. The arithmetic mean of these + * would be heavily biased towards the 10. + * 2) Absolute cost (cost = weight * rowCount + (1 - weight) * size) - when adding up + * two numeric measurements that have different units we can easily end up with one + * overwhelming the other. + */ def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { - 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 + 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) + Math.pow(relativeRows.doubleValue, conf.joinReorderCardWeight) * + Math.pow(relativeSize.doubleValue, 1 - conf.joinReorderCardWeight) < 1 + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d91bb59f68c44..7138c56560a3f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1926,8 +1926,10 @@ object SQLConf { val JOIN_REORDER_CARD_WEIGHT = buildConf("spark.sql.cbo.joinReorder.card.weight") .internal() - .doc("The weight of cardinality (number of rows) for plan cost comparison in join reorder: " + - "rows * weight + size * (1 - weight).") + .doc("The weight of the ratio of cardinalities (number of rows) " + + "in the cost comparison function. The ratio of sizes in bytes has weight " + + "1 - this value. The weighted geometric mean of these ratios is used to decide " + + "which of the candidate plans will be chosen by the CBO.") .version("2.2.0") .doubleConf .checkValue(weight => weight >= 0 && weight <= 1, "The weight value must be in [0, 1].") 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 f41c6753f4f74..f5e2ff4f1b3e8 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 @@ -369,9 +369,6 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB 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)) 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 a42914765dcc8..c4a660532b207 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,13 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs (nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) val expected = - t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) + 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("d1_c2") === 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-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index c2909d9f5531f..d7a40fad9ce61 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -23,7 +23,7 @@ : : : +- * Project (8) : : : +- * Filter (7) : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.household_demographics (5) + : : : +- Scan parquet default.date_dim (5) : : +- * Project (15) : : +- * Filter (14) : : +- * ColumnarToRow (13) @@ -31,7 +31,7 @@ : +- * Project (22) : +- * Filter (21) : +- * ColumnarToRow (20) - : +- Scan parquet default.date_dim (19) + : +- Scan parquet default.household_demographics (19) +- * Filter (32) +- * ColumnarToRow (31) +- Scan parquet default.customer (30) @@ -54,96 +54,96 @@ Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnu (4) BroadcastExchange Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[4, int, true] as bigint)),false), [id=#6] -(5) Scan parquet default.household_demographics -Output [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] +(5) Scan parquet default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1998,1999,2000]), In(d_date_sk, [2451790,2451119,2451180,2451454,2450874,2450906,2450967,2451485,2451850,2451514,2451270,2451758,2451028,2451546,2450997,2450996,2451393,2451667,2451453,2451819,2450905,2451331,2451577,2451089,2451301,2451545,2451605,2451851,2451181,2451149,2451820,2451362,2451392,2451240,2450935,2451637,2451484,2451058,2451300,2451727,2451759,2450815,2451698,2451150,2451332,2451606,2451666,2451211,2450846,2450875,2450966,2450936,2451361,2451212,2451880,2451059,2451789,2451423,2451576,2450816,2451088,2451728,2451027,2451120,2451881,2451697,2450847,2451271,2451636,2451515,2451424,2451239]), IsNotNull(d_date_sk)] +ReadSchema: struct (6) ColumnarToRow -Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] (7) Filter -Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] -Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000 ) OR (hd_buy_potential#8 = Unknown ))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1998,1999,2000)) AND d_date_sk#7 INSET (2451790,2451119,2451180,2451454,2450874,2450906,2450967,2451485,2451850,2451514,2451270,2451758,2451028,2451546,2450997,2450996,2451393,2451667,2451453,2451819,2450905,2451331,2451577,2451089,2451301,2451545,2451605,2451851,2451181,2451149,2451820,2451362,2451392,2451240,2450935,2451637,2451484,2451058,2451300,2451727,2451759,2450815,2451698,2451150,2451332,2451606,2451666,2451211,2450846,2450875,2450966,2450936,2451361,2451212,2451880,2451059,2451789,2451423,2451576,2450816,2451088,2451728,2451027,2451120,2451881,2451697,2450847,2451271,2451636,2451515,2451424,2451239)) AND isnotnull(d_date_sk#7)) (8) Project -Output [1]: [hd_demo_sk#7] -Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] +Output [1]: [d_date_sk#7] +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] (9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#7] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] Join condition: None (10) Project [codegen id : 2] -Output [4]: [ss_customer_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, hd_demo_sk#7] +Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] (11) BroadcastExchange -Input [4]: [ss_customer_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#11] +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#10] (12) Scan parquet default.store -Output [2]: [s_store_sk#12, s_county#13] +Output [2]: [s_store_sk#11, s_county#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_county, [Fairfield County,Ziebach County,Bronx County,Barrow County]), IsNotNull(s_store_sk)] ReadSchema: struct (13) ColumnarToRow -Input [2]: [s_store_sk#12, s_county#13] +Input [2]: [s_store_sk#11, s_county#12] (14) Filter -Input [2]: [s_store_sk#12, s_county#13] -Condition : (s_county#13 IN (Fairfield County,Ziebach County,Bronx County,Barrow County) AND isnotnull(s_store_sk#12)) +Input [2]: [s_store_sk#11, s_county#12] +Condition : (s_county#12 IN (Fairfield County,Ziebach County,Bronx County,Barrow County) AND isnotnull(s_store_sk#11)) (15) Project -Output [1]: [s_store_sk#12] -Input [2]: [s_store_sk#12, s_county#13] +Output [1]: [s_store_sk#11] +Input [2]: [s_store_sk#11, s_county#12] (16) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Right keys [1]: [s_store_sk#11] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_customer_sk#1, ss_ticket_number#4, ss_sold_date_sk#5] -Input [5]: [ss_customer_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, s_store_sk#12] +Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#11] (18) BroadcastExchange -Input [3]: [ss_customer_sk#1, ss_ticket_number#4, ss_sold_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#14] +Input [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#13] -(19) Scan parquet default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_dom#17] +(19) Scan parquet default.household_demographics +Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1998,1999,2000]), In(d_date_sk, [2451790,2451119,2451180,2451454,2450874,2450906,2450967,2451485,2451850,2451514,2451270,2451758,2451028,2451546,2450997,2450996,2451393,2451667,2451453,2451819,2450905,2451331,2451577,2451089,2451301,2451545,2451605,2451851,2451181,2451149,2451820,2451362,2451392,2451240,2450935,2451637,2451484,2451058,2451300,2451727,2451759,2450815,2451698,2451150,2451332,2451606,2451666,2451211,2450846,2450875,2450966,2450936,2451361,2451212,2451880,2451059,2451789,2451423,2451576,2450816,2451088,2451728,2451027,2451120,2451881,2451697,2450847,2451271,2451636,2451515,2451424,2451239]), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +ReadSchema: struct (20) ColumnarToRow -Input [3]: [d_date_sk#15, d_year#16, d_dom#17] +Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] (21) Filter -Input [3]: [d_date_sk#15, d_year#16, d_dom#17] -Condition : (((((isnotnull(d_dom#17) AND (d_dom#17 >= 1)) AND (d_dom#17 <= 2)) AND d_year#16 IN (1998,1999,2000)) AND d_date_sk#15 INSET (2451790,2451119,2451180,2451454,2450874,2450906,2450967,2451485,2451850,2451514,2451270,2451758,2451028,2451546,2450997,2450996,2451393,2451667,2451453,2451819,2450905,2451331,2451577,2451089,2451301,2451545,2451605,2451851,2451181,2451149,2451820,2451362,2451392,2451240,2450935,2451637,2451484,2451058,2451300,2451727,2451759,2450815,2451698,2451150,2451332,2451606,2451666,2451211,2450846,2450875,2450966,2450936,2451361,2451212,2451880,2451059,2451789,2451423,2451576,2450816,2451088,2451728,2451027,2451120,2451881,2451697,2450847,2451271,2451636,2451515,2451424,2451239)) AND isnotnull(d_date_sk#15)) +Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (22) Project -Output [1]: [d_date_sk#15] -Input [3]: [d_date_sk#15, d_year#16, d_dom#17] +Output [1]: [hd_demo_sk#14] +Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] (23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#14] Join condition: None (24) Project [codegen id : 4] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#15] +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#14] (25) HashAggregate [codegen id : 4] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt index 67d569f0d1134..91deecbd03317 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt @@ -15,17 +15,17 @@ WholeStageCodegen (7) WholeStageCodegen (4) HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [ss_customer_sk,ss_ticket_number,ss_sold_date_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] BroadcastHashJoin [ss_store_sk,s_store_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Project [ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) @@ -33,21 +33,21 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - Project [hd_demo_sk] - Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] + Project [d_date_sk] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] Project [s_store_sk] Filter [s_county,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county] - Project [d_date_sk] - Filter [d_dom,d_year,d_date_sk] + Project [hd_demo_sk] + Filter [hd_vehicle_count,hd_dep_count,hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dom] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] Filter [c_customer_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index 039a38acb9f22..7c414d9080734 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.web_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.web_sales @@ -36,120 +36,120 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#5] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] +(5) Sort [codegen id : 2] +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Condition : (i_category#11 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#6)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#12] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#1, ws_ext_sales_price#2] -Input [4]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, d_date_sk#5] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#1, ws_ext_sales_price#2] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#8] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#1, ws_ext_sales_price#2] -Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(13) Scan parquet default.item -Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_date#14] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] +Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 1999-02-22)) AND (d_date#14 <= 1999-03-24)) AND isnotnull(d_date_sk#13)) -(16) Exchange -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#15] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#13] +Input [2]: [d_date_sk#13, d_date#14] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#9] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#13] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Input [8]: [ws_item_sk#1, ws_ext_sales_price#2, i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +Output [6]: [ws_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11, d_date_sk#13] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [ws_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum#16] -Results [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] +Results [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] (21) Exchange -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Arguments: hashpartitioning(i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Arguments: hashpartitioning(i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, 5), ENSURE_REQUIREMENTS, [id=#18] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#19] -Results [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w1#22, i_item_id#10] +Results [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w1#22, i_item_id#7] (23) Exchange -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: hashpartitioning(i_class#13, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: hashpartitioning(i_class#10, 5), ENSURE_REQUIREMENTS, [id=#23] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: [i_class#13 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: [i_class#10 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: [sum(_w1#22) windowspecdefinition(i_class#13, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#13] +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: [sum(_w1#22) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#10] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25, i_item_id#10] -Input [9]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10, _we0#24] +Output [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25, i_item_id#7] +Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7, _we0#24] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25, i_item_id#10] -Arguments: 100, [i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] +Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25, i_item_id#7] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] ===== Subqueries ===== @@ -157,7 +157,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dyn ReusedExchange (28) -(28) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(28) ReusedExchange [Reuses operator id: 17] +Output [1]: [d_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt index 31f5c92093804..e981328e2d9b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt @@ -14,36 +14,36 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ws_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #4 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 55dca59270306..7d562331218b9 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 (29) - : +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) : :- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (15) @@ -27,16 +27,16 @@ : : +- * Project (19) : : +- * Filter (18) : : +- * ColumnarToRow (17) - : : +- Scan parquet default.customer_address (16) - : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.date_dim (23) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.store (23) +- BroadcastExchange (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.store (30) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.customer_address (29) (1) Scan parquet default.store_sales @@ -108,94 +108,94 @@ Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = A Output [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Input [13]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14, hd_demo_sk#16, hd_dep_count#17] -(16) Scan parquet default.customer_address -Output [3]: [ca_address_sk#19, ca_state#20, ca_country#21] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#19, d_year#20] 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 +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct (17) ColumnarToRow [codegen id : 3] -Input [3]: [ca_address_sk#19, ca_state#20, ca_country#21] +Input [2]: [d_date_sk#19, d_year#20] (18) Filter [codegen id : 3] -Input [3]: [ca_address_sk#19, ca_state#20, ca_country#21] -Condition : (((isnotnull(ca_country#21) AND (ca_country#21 = United States)) AND isnotnull(ca_address_sk#19)) AND ((ca_state#20 IN (TX,OH) OR ca_state#20 IN (OR,NM,KY)) OR ca_state#20 IN (VA,TX,MS))) +Input [2]: [d_date_sk#19, d_year#20] +Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) (19) Project [codegen id : 3] -Output [2]: [ca_address_sk#19, ca_state#20] -Input [3]: [ca_address_sk#19, ca_state#20, ca_country#21] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_year#20] (20) BroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#19] -Join condition: ((((ca_state#20 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#20 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#20 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) +Left keys [1]: [ss_sold_date_sk#10] +Right keys [1]: [d_date_sk#19] +Join condition: None (22) Project [codegen id : 6] -Output [5]: [ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Input [9]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#19, ca_state#20] +Output [6]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9] +Input [8]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, d_date_sk#19] -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#23, d_year#24] +(23) Scan parquet default.store +Output [1]: [s_store_sk#22] 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}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#23, d_year#24] +Input [1]: [s_store_sk#22] (25) Filter [codegen id : 4] -Input [2]: [d_date_sk#23, d_year#24] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_sk#23)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#23] -Input [2]: [d_date_sk#23, d_year#24] +Input [1]: [s_store_sk#22] +Condition : isnotnull(s_store_sk#22) -(27) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +(26) BroadcastExchange +Input [1]: [s_store_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#23] +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#22] Join condition: None -(29) Project [codegen id : 6] -Output [4]: [ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [6]: [ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#23] +(28) Project [codegen id : 6] +Output [5]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9] +Input [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, s_store_sk#22] -(30) Scan parquet default.store -Output [1]: [s_store_sk#26] +(29) Scan parquet default.customer_address +Output [3]: [ca_address_sk#24, ca_state#25, ca_country#26] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_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 + +(30) ColumnarToRow [codegen id : 5] +Input [3]: [ca_address_sk#24, ca_state#25, ca_country#26] -(31) ColumnarToRow [codegen id : 5] -Input [1]: [s_store_sk#26] +(31) Filter [codegen id : 5] +Input [3]: [ca_address_sk#24, ca_state#25, ca_country#26] +Condition : (((isnotnull(ca_country#26) AND (ca_country#26 = United States)) AND isnotnull(ca_address_sk#24)) AND ((ca_state#25 IN (TX,OH) OR ca_state#25 IN (OR,NM,KY)) OR ca_state#25 IN (VA,TX,MS))) -(32) Filter [codegen id : 5] -Input [1]: [s_store_sk#26] -Condition : isnotnull(s_store_sk#26) +(32) Project [codegen id : 5] +Output [2]: [ca_address_sk#24, ca_state#25] +Input [3]: [ca_address_sk#24, ca_state#25, ca_country#26] (33) BroadcastExchange -Input [1]: [s_store_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#26] -Join condition: None +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#24] +Join condition: ((((ca_state#25 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#25 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#25 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [5]: [ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, s_store_sk#26] +Input [7]: [ss_addr_sk#3, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ca_address_sk#24, ca_state#25] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] @@ -221,7 +221,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dy ReusedExchange (39) -(39) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#23] +(39) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#19] 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 e198650feb1db..26be2b52e400a 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_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_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] Project [ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,cd_marital_status,cd_education_status] @@ -35,25 +35,25 @@ WholeStageCodegen (7) InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] InputAdapter - BroadcastExchange #5 + BroadcastExchange #2 WholeStageCodegen (3) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #2 + BroadcastExchange #5 WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] + 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] 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 85aed12cc9d4f..f7927aad003a2 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) - +- * 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) + +- * 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) (1) Scan parquet default.catalog_sales @@ -173,89 +173,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 : 6] +(27) ColumnarToRow [codegen id : 7] Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(28) Filter [codegen id : 6] +(28) Filter [codegen id : 7] Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Condition : (((c_birth_month#26 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) -(29) Project [codegen id : 6] +(29) Project [codegen id : 7] Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(30) Exchange -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#28] - -(31) Sort [codegen id : 7] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#29] +(30) Scan parquet default.customer_address +Output [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [1]: [cd_demo_sk#29] +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 -(34) Filter [codegen id : 8] -Input [1]: [cd_demo_sk#29] -Condition : isnotnull(cd_demo_sk#29) +(31) ColumnarToRow [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(35) Exchange -Input [1]: [cd_demo_sk#29] -Arguments: hashpartitioning(cd_demo_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] +(32) Filter [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Condition : (ca_state#30 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#28)) -(36) Sort [codegen id : 9] -Input [1]: [cd_demo_sk#29] -Arguments: [cd_demo_sk#29 ASC NULLS FIRST], false, 0 +(33) BroadcastExchange +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(37) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#29] +(34) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#28] Join condition: None -(38) Project [codegen id : 11] -Output [3]: [c_customer_sk#23, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, cd_demo_sk#29] +(35) Project [codegen id : 7] +Output [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [8]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] +(36) Exchange +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#33] + +(37) Sort [codegen id : 8] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 + +(38) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#34] 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 +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] +(39) ColumnarToRow [codegen id : 9] +Input [1]: [cd_demo_sk#34] -(41) Filter [codegen id : 10] -Input [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#31)) +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(42) BroadcastExchange -Input [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#35] +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 + +(43) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#34] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] -Input [7]: [c_customer_sk#23, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] +Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31, cd_demo_sk#34] (45) Exchange -Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -264,12 +264,12 @@ Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#34, ca_state#33, ca_county#32] -Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] +Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] (49) Expand [codegen id : 13] -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#34, ca_state#33, ca_county#32] -Arguments: [List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#34, ca_state#33, ca_county#32, 0), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#34, ca_state#33, null, 1), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#34, null, null, 3), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, null, null, null, 7), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, null, null, null, null, 15)], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Arguments: [List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29, 0), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, null, 1), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, null, null, 3), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, null, null, null, 7), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, null, null, null, null, 15)], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] (50) HashAggregate [codegen id : 13] Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] 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 64997d52f07a3..29f977c87cb10 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 @@ -56,34 +56,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] - 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) + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (8) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (7) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - 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 + 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 - 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 (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] 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 b1080611d7eee..febae3b0df780 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,259 +4,259 @@ TakeOrderedAndProject (45) +- Exchange (43) +- * HashAggregate (42) +- * Project (41) - +- * 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] + +- * 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_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_sold_date_sk#5] 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 +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_sold_date_sk#5] + +(3) Filter [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_sold_date_sk#5] +Condition : ((isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, i_manager_id#12] +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#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, i_manager_id#12] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, i_manager_id#12] +Condition : ((isnotnull(i_manager_id#12) AND (i_manager_id#12 = 8)) AND isnotnull(i_item_sk#7)) -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(7) Project [codegen id : 1] +Output [5]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] +Input [6]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, i_manager_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)) +(8) BroadcastExchange +Input [5]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(10) Project [codegen id : 4] +Output [8]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_sold_date_sk#5, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] -(6) Scan parquet default.store_sales -Output [5]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, ss_sold_date_sk#9] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#14, d_year#15, d_moy#16] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_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,1998), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] + +(13) Filter [codegen id : 2] +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Condition : ((((isnotnull(d_moy#16) AND isnotnull(d_year#15)) AND (d_moy#16 = 11)) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(7) ColumnarToRow -Input [5]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, ss_sold_date_sk#9] +(14) Project [codegen id : 2] +Output [1]: [d_date_sk#14] +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -(8) Filter -Input [5]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, ss_sold_date_sk#9] -Condition : ((isnotnull(ss_item_sk#5) AND isnotnull(ss_customer_sk#6)) AND isnotnull(ss_store_sk#7)) +(15) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#9] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None -(10) Project [codegen id : 3] -Output [4]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8] -Input [6]: [d_date_sk#1, ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, ss_sold_date_sk#9] +(17) Project [codegen id : 4] +Output [7]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] +Input [9]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, ss_sold_date_sk#5, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, d_date_sk#14] -(11) Scan parquet default.store -Output [2]: [s_store_sk#11, s_zip#12] +(18) Scan parquet default.store +Output [2]: [s_store_sk#18, s_zip#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_zip#12] +(19) ColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#18, s_zip#19] -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#11, s_zip#12] -Condition : (isnotnull(s_zip#12) AND isnotnull(s_store_sk#11)) +(20) Filter [codegen id : 3] +Input [2]: [s_store_sk#18, s_zip#19] +Condition : (isnotnull(s_zip#19) AND isnotnull(s_store_sk#18)) -(14) BroadcastExchange -Input [2]: [s_store_sk#11, s_zip#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] +(21) BroadcastExchange +Input [2]: [s_store_sk#18, s_zip#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#11] +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None -(16) Project [codegen id : 3] -Output [4]: [ss_item_sk#5, ss_customer_sk#6, ss_ext_sales_price#8, s_zip#12] -Input [6]: [ss_item_sk#5, ss_customer_sk#6, ss_store_sk#7, ss_ext_sales_price#8, s_store_sk#11, s_zip#12] +(23) Project [codegen id : 4] +Output [7]: [ss_customer_sk#2, ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, s_zip#19] +Input [9]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, s_store_sk#18, s_zip#19] -(17) Exchange -Input [4]: [ss_item_sk#5, ss_customer_sk#6, ss_ext_sales_price#8, s_zip#12] -Arguments: hashpartitioning(ss_customer_sk#6, 5), ENSURE_REQUIREMENTS, [id=#14] +(24) Exchange +Input [7]: [ss_customer_sk#2, ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, s_zip#19] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] -(18) Sort [codegen id : 4] -Input [4]: [ss_item_sk#5, ss_customer_sk#6, ss_ext_sales_price#8, s_zip#12] -Arguments: [ss_customer_sk#6 ASC NULLS FIRST], false, 0 +(25) Sort [codegen id : 5] +Input [7]: [ss_customer_sk#2, ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, s_zip#19] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(19) Scan parquet default.customer -Output [2]: [c_customer_sk#15, c_current_addr_sk#16] +(26) Scan parquet default.customer +Output [2]: [c_customer_sk#22, c_current_addr_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] +(27) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -(21) Filter [codegen id : 5] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) +(28) Filter [codegen id : 6] +Input [2]: [c_customer_sk#22, c_current_addr_sk#23] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) -(22) Exchange -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: hashpartitioning(c_current_addr_sk#16, 5), ENSURE_REQUIREMENTS, [id=#17] +(29) Exchange +Input [2]: [c_customer_sk#22, c_current_addr_sk#23] +Arguments: hashpartitioning(c_current_addr_sk#23, 5), ENSURE_REQUIREMENTS, [id=#24] -(23) Sort [codegen id : 6] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: [c_current_addr_sk#16 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 7] +Input [2]: [c_customer_sk#22, c_current_addr_sk#23] +Arguments: [c_current_addr_sk#23 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#18, ca_zip#19] +(31) Scan parquet default.customer_address +Output [2]: [ca_address_sk#25, ca_zip#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_zip#19] +(32) ColumnarToRow [codegen id : 8] +Input [2]: [ca_address_sk#25, ca_zip#26] -(26) Filter [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_zip#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_zip#19)) +(33) Filter [codegen id : 8] +Input [2]: [ca_address_sk#25, ca_zip#26] +Condition : (isnotnull(ca_address_sk#25) AND isnotnull(ca_zip#26)) -(27) Exchange -Input [2]: [ca_address_sk#18, ca_zip#19] -Arguments: hashpartitioning(ca_address_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +(34) Exchange +Input [2]: [ca_address_sk#25, ca_zip#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#18, ca_zip#19] -Arguments: [ca_address_sk#18 ASC NULLS FIRST], false, 0 +(35) Sort [codegen id : 9] +Input [2]: [ca_address_sk#25, ca_zip#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#18] +(36) SortMergeJoin [codegen id : 10] +Left keys [1]: [c_current_addr_sk#23] +Right keys [1]: [ca_address_sk#25] Join condition: None -(30) Project [codegen id : 9] -Output [2]: [c_customer_sk#15, ca_zip#19] -Input [4]: [c_customer_sk#15, c_current_addr_sk#16, ca_address_sk#18, ca_zip#19] - -(31) Exchange -Input [2]: [c_customer_sk#15, ca_zip#19] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] - -(32) Sort [codegen id : 10] -Input [2]: [c_customer_sk#15, ca_zip#19] -Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 +(37) Project [codegen id : 10] +Output [2]: [c_customer_sk#22, ca_zip#26] +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, ca_address_sk#25, ca_zip#26] -(33) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#6] -Right keys [1]: [c_customer_sk#15] -Join condition: NOT (substr(ca_zip#19, 1, 5) = substr(s_zip#12, 1, 5)) +(38) Exchange +Input [2]: [c_customer_sk#22, ca_zip#26] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#28] -(34) Project [codegen id : 12] -Output [2]: [ss_item_sk#5, ss_ext_sales_price#8] -Input [6]: [ss_item_sk#5, ss_customer_sk#6, ss_ext_sales_price#8, s_zip#12, c_customer_sk#15, ca_zip#19] +(39) Sort [codegen id : 11] +Input [2]: [c_customer_sk#22, ca_zip#26] +Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.item -Output [6]: [i_item_sk#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26, i_manager_id#27] -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#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26, i_manager_id#27] - -(37) Filter [codegen id : 11] -Input [6]: [i_item_sk#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26, i_manager_id#27] -Condition : ((isnotnull(i_manager_id#27) AND (i_manager_id#27 = 8)) AND isnotnull(i_item_sk#22)) - -(38) Project [codegen id : 11] -Output [5]: [i_item_sk#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26] -Input [6]: [i_item_sk#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26, i_manager_id#27] - -(39) BroadcastExchange -Input [5]: [i_item_sk#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] - -(40) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#22] -Join condition: None +(40) SortMergeJoin [codegen id : 12] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#22] +Join condition: NOT (substr(ca_zip#26, 1, 5) = substr(s_zip#19, 1, 5)) (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#8, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26] -Input [7]: [ss_item_sk#5, ss_ext_sales_price#8, i_item_sk#22, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26] +Output [5]: [ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] +Input [9]: [ss_customer_sk#2, ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11, s_zip#19, c_customer_sk#22, ca_zip#26] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#8, i_brand_id#23, i_brand#24, i_manufact_id#25, i_manufact#26] -Keys [4]: [i_brand#24, i_brand_id#23, i_manufact_id#25, i_manufact#26] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#8))] +Input [5]: [ss_ext_sales_price#4, i_brand_id#8, i_brand#9, i_manufact_id#10, i_manufact#11] +Keys [4]: [i_brand#9, i_brand_id#8, i_manufact_id#10, i_manufact#11] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum#29] -Results [5]: [i_brand#24, i_brand_id#23, i_manufact_id#25, i_manufact#26, sum#30] +Results [5]: [i_brand#9, i_brand_id#8, i_manufact_id#10, i_manufact#11, sum#30] (43) Exchange -Input [5]: [i_brand#24, i_brand_id#23, i_manufact_id#25, i_manufact#26, sum#30] -Arguments: hashpartitioning(i_brand#24, i_brand_id#23, i_manufact_id#25, i_manufact#26, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [5]: [i_brand#9, i_brand_id#8, i_manufact_id#10, i_manufact#11, sum#30] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, i_manufact_id#10, i_manufact#11, 5), ENSURE_REQUIREMENTS, [id=#31] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#24, i_brand_id#23, i_manufact_id#25, i_manufact#26, sum#30] -Keys [4]: [i_brand#24, i_brand_id#23, i_manufact_id#25, i_manufact#26] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#8))#32] -Results [5]: [i_brand_id#23 AS brand_id#33, i_brand#24 AS brand#34, i_manufact_id#25, i_manufact#26, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#8))#32,17,2) AS ext_price#35] +Input [5]: [i_brand#9, i_brand_id#8, i_manufact_id#10, i_manufact#11, sum#30] +Keys [4]: [i_brand#9, i_brand_id#8, i_manufact_id#10, i_manufact#11] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#32] +Results [5]: [i_brand_id#8 AS brand_id#33, i_brand#9 AS brand#34, i_manufact_id#10, i_manufact#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#32,17,2) AS ext_price#35] (45) TakeOrderedAndProject -Input [5]: [brand_id#33, brand#34, i_manufact_id#25, i_manufact#26, ext_price#35] -Arguments: 100, [ext_price#35 DESC NULLS LAST, brand#34 ASC NULLS FIRST, brand_id#33 ASC NULLS FIRST, i_manufact_id#25 ASC NULLS FIRST, i_manufact#26 ASC NULLS FIRST], [brand_id#33, brand#34, i_manufact_id#25, i_manufact#26, ext_price#35] +Input [5]: [brand_id#33, brand#34, i_manufact_id#10, i_manufact#11, ext_price#35] +Arguments: 100, [ext_price#35 DESC NULLS LAST, brand#34 ASC NULLS FIRST, brand_id#33 ASC NULLS FIRST, i_manufact_id#10 ASC NULLS FIRST, i_manufact#11 ASC NULLS FIRST], [brand_id#33, brand#34, i_manufact_id#10, i_manufact#11, ext_price#35] ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 ReusedExchange (46) -(46) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#1] +(46) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#14] 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 a10582e55b384..a563ccd2a0106 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,73 +6,73 @@ 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] - 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 [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] + 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_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_customer_sk,ss_store_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - 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] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + 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.customer [c_customer_sk,c_current_addr_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter - WholeStageCodegen (8) - Sort [ca_address_sk] + BroadcastExchange #3 + 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 InputAdapter - 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] + Scan parquet default.store [s_store_sk,s_zip] InputAdapter - 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] + 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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index a40286f9f8f83..d6bbbfc515205 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.catalog_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.catalog_sales @@ -36,120 +36,120 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#5] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] +(5) Sort [codegen id : 2] +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Condition : (i_category#11 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#6)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#12] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#1, cs_ext_sales_price#2] -Input [4]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, d_date_sk#5] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#1, cs_ext_sales_price#2] -Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#8] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#1, cs_ext_sales_price#2] -Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(13) Scan parquet default.item -Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_date#14] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] +Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 1999-02-22)) AND (d_date#14 <= 1999-03-24)) AND isnotnull(d_date_sk#13)) -(16) Exchange -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#15] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#13] +Input [2]: [d_date_sk#13, d_date#14] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#9] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#13] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Input [8]: [cs_item_sk#1, cs_ext_sales_price#2, i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +Output [6]: [cs_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11, d_date_sk#13] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [cs_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum#16] -Results [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] +Results [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] (21) Exchange -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Arguments: hashpartitioning(i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Arguments: hashpartitioning(i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, 5), ENSURE_REQUIREMENTS, [id=#18] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#19] -Results [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w1#22, i_item_id#10] +Results [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w1#22, i_item_id#7] (23) Exchange -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: hashpartitioning(i_class#13, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: hashpartitioning(i_class#10, 5), ENSURE_REQUIREMENTS, [id=#23] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: [i_class#13 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: [i_class#10 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: [sum(_w1#22) windowspecdefinition(i_class#13, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#13] +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: [sum(_w1#22) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#10] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25, i_item_id#10] -Input [9]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10, _we0#24] +Output [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25, i_item_id#7] +Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7, _we0#24] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25, i_item_id#10] -Arguments: 100, [i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] +Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25, i_item_id#7] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] ===== Subqueries ===== @@ -157,7 +157,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dyn ReusedExchange (28) -(28) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(28) ReusedExchange [Reuses operator id: 17] +Output [1]: [d_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt index c8dca9b614245..bc14161be8ee2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt @@ -14,36 +14,36 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (2) - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #4 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 7895ee1838ad7..351c7f0024dd0 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 @@ -320,12 +320,12 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (67) : : : +- * Filter (66) : : : +- * ColumnarToRow (65) - : : : +- Scan parquet default.customer (64) + : : : +- Scan parquet default.item (64) : : +- * Sort (77) : : +- Exchange (76) : : +- * Filter (75) : : +- * ColumnarToRow (74) - : : +- Scan parquet default.item (73) + : : +- Scan parquet default.customer (73) : +- * Sort (86) : +- Exchange (85) : +- * Filter (84) @@ -390,89 +390,89 @@ Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#5 (62) Exchange Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 -(64) Scan parquet default.customer -Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +(64) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] 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 (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) (67) Exchange -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (68) Sort [codegen id : 5] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (71) Exchange -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(73) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +(73) Scan parquet default.customer +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] 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 (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (75) Filter [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) (76) Exchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address Output [3]: [ca_state#77, ca_zip#78, ca_country#79] @@ -497,20 +497,20 @@ Input [3]: [ca_state#77, ca_zip#78, ca_country#79] Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#66, s_zip#60] +Left keys [2]: [c_birth_country#74, s_zip#60] Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns @@ -545,23 +545,23 @@ Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] (100) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] 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 659984c3b6a70..7c896c0838a85 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,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] + 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] 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,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] + 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] InputAdapter WholeStageCodegen (7) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - 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] + 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] InputAdapter WholeStageCodegen (3) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #15 + Exchange [ss_item_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] @@ -62,24 +62,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [i_item_sk] #17 WholeStageCodegen (4) - 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 (9) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #18 + Exchange [c_customer_sk] #18 WholeStageCodegen (8) - 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 (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 e45459d529104..97ee167a14b5a 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 @@ -320,12 +320,12 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (67) : : : +- * Filter (66) : : : +- * ColumnarToRow (65) - : : : +- Scan parquet default.customer (64) + : : : +- Scan parquet default.item (64) : : +- * Sort (77) : : +- Exchange (76) : : +- * Filter (75) : : +- * ColumnarToRow (74) - : : +- Scan parquet default.item (73) + : : +- Scan parquet default.customer (73) : +- * Sort (86) : +- Exchange (85) : +- * Filter (84) @@ -390,89 +390,89 @@ Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#5 (62) Exchange Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 -(64) Scan parquet default.customer -Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +(64) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] 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 (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) (67) Exchange -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (68) Sort [codegen id : 5] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] (71) Exchange -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(73) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +(73) Scan parquet default.customer +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] 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 (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (75) Filter [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) (76) Exchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] (80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address Output [3]: [ca_state#77, ca_zip#78, ca_country#79] @@ -497,20 +497,20 @@ Input [3]: [ca_state#77, ca_zip#78, ca_country#79] Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#66, s_zip#60] +Left keys [2]: [c_birth_country#74, s_zip#60] Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns @@ -545,23 +545,23 @@ Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] (100) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] 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 659984c3b6a70..7c896c0838a85 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,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] + 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] 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,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] + 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] InputAdapter WholeStageCodegen (7) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - 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] + 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] InputAdapter WholeStageCodegen (3) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #15 + Exchange [ss_item_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] @@ -62,24 +62,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [i_item_sk] #17 WholeStageCodegen (4) - 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 (9) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #18 + Exchange [c_customer_sk] #18 WholeStageCodegen (8) - 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 (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 1fa837bcf3e26..8ebdf815e265c 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 (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 (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 (51) +- Exchange (50) +- * Project (49) - +- * 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) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Filter (46) + : +- * ColumnarToRow (45) + : +- Scan parquet default.catalog_sales (44) + +- ReusedExchange (47) (1) Scan parquet default.store_sales @@ -226,77 +226,77 @@ Input [6]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#2 (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, 5), ENSURE_REQUIREMENTS, [id=#32] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#32] (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#2 as bigint), cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#4 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#5, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] + +(42) Exchange +Input [8]: [ss_net_profit#5, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#33] + +(43) Sort [codegen id : 13] +Input [8]: [ss_net_profit#5, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19, 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 -(40) Scan parquet default.catalog_sales -Output [4]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, cs_sold_date_sk#36] +(44) Scan parquet default.catalog_sales +Output [4]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36, cs_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#36), dynamicpruningexpression(cs_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [4]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, cs_sold_date_sk#36] - -(42) Filter [codegen id : 13] -Input [4]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, cs_sold_date_sk#36] -Condition : (isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) +(45) ColumnarToRow [codegen id : 15] +Input [4]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36, cs_sold_date_sk#37] -(43) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#38] +(46) Filter [codegen id : 15] +Input [4]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36, cs_sold_date_sk#37] +Condition : (isnotnull(cs_bill_customer_sk#34) AND isnotnull(cs_item_sk#35)) -(44) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#36] -Right keys [1]: [d_date_sk#38] -Join condition: None - -(45) Project [codegen id : 13] -Output [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Input [5]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, cs_sold_date_sk#36, d_date_sk#38] - -(46) 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), ENSURE_REQUIREMENTS, [id=#39] +(47) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#39] -(47) Sort [codegen id : 14] -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 - -(48) SortMergeJoin [codegen id : 15] -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)] +(48) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join condition: None (49) Project [codegen id : 15] -Output [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#35] -Input [7]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Output [3]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36] +Input [5]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#39] (50) Exchange -Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#35] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [3]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#34 as bigint), cast(cs_item_sk#35 as bigint), 5), ENSURE_REQUIREMENTS, [id=#40] (51) Sort [codegen id : 16] -Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#35] -Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 +Input [3]: [cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36] +Arguments: [cast(cs_bill_customer_sk#34 as bigint) ASC NULLS FIRST, cast(cs_item_sk#35 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [3]: [cast(ss_customer_sk#2 as bigint), cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#4 as bigint)] -Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] +Right keys [2]: [cast(cs_bill_customer_sk#34 as bigint), cast(cs_item_sk#35 as bigint)] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#5, sr_net_loss#25, cs_net_profit#35, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#35] +Output [7]: [ss_net_profit#5, sr_net_loss#25, cs_net_profit#36, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19] +Input [11]: [ss_net_profit#5, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#34, cs_item_sk#35, cs_net_profit#36] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#5, sr_net_loss#25, cs_net_profit#35, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19] +Input [7]: [ss_net_profit#5, sr_net_loss#25, cs_net_profit#36, s_store_id#13, s_store_name#14, i_item_id#18, i_item_desc#19] Keys [4]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#36))] Aggregate Attributes [3]: [sum#41, sum#42, sum#43] Results [7]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14, sum#44, sum#45, sum#46] @@ -307,9 +307,9 @@ Arguments: hashpartitioning(i_item_id#18, i_item_desc#19, s_store_id#13, s_store (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14, sum#44, sum#45, sum#46] Keys [4]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#48, sum(UnscaledValue(sr_net_loss#25))#49, sum(UnscaledValue(cs_net_profit#35))#50] -Results [7]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#48,17,2) AS store_sales_profit#51, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#49,17,2) AS store_returns_loss#52, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#50,17,2) AS catalog_sales_profit#53] +Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#36))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#48, sum(UnscaledValue(sr_net_loss#25))#49, sum(UnscaledValue(cs_net_profit#36))#50] +Results [7]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#48,17,2) AS store_sales_profit#51, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#49,17,2) AS store_returns_loss#52, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#50,17,2) AS catalog_sales_profit#53] (57) TakeOrderedAndProject Input [7]: [i_item_id#18, i_item_desc#19, s_store_id#13, s_store_name#14, store_sales_profit#51, store_returns_loss#52, catalog_sales_profit#53] @@ -331,11 +331,11 @@ ReusedExchange (59) (59) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#28] -Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#36 IN dynamicpruning#37 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#38 ReusedExchange (60) (60) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#39] 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 f72b5a4c229b6..60358dea7025b 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,69 +6,69 @@ 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 [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (13) + Sort [sr_customer_sk,sr_item_sk] InputAdapter - 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] + 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] InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - 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_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #4 - 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] + 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 - 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 + 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_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + 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 - 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] + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #7 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk] + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #8 + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -86,19 +86,19 @@ 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] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + ReusedExchange [d_date_sk] #9 InputAdapter - 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] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - ReusedExchange [d_date_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #9 + 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 eeda8611876d6..2aa99626920ec 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 (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * 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 (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer_address (11) + : : +- 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 (27) - : +- * 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) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.customer_address (23) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -114,108 +114,108 @@ Join condition: None Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -(11) Scan parquet default.customer_address -Output [2]: [ca_address_sk#10, ca_gmt_offset#11] -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#10, ca_gmt_offset#11] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#10, ca_gmt_offset#11] -Condition : ((isnotnull(ca_gmt_offset#11) AND (ca_gmt_offset#11 = -5.00)) AND isnotnull(ca_address_sk#10)) - -(14) Project [codegen id : 2] -Output [1]: [ca_address_sk#10] -Input [2]: [ca_address_sk#10, ca_gmt_offset#11] - -(15) BroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#10] -Join condition: None - -(17) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#13, i_manufact_id#14] +(11) Scan parquet default.item +Output [2]: [i_item_sk#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_manufact_id#14] +(12) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#10, i_manufact_id#11] -(20) Filter [codegen id : 4] -Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : isnotnull(i_item_sk#13) +(13) Filter [codegen id : 3] +Input [2]: [i_item_sk#10, i_manufact_id#11] +Condition : isnotnull(i_item_sk#10) -(21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#16] +(14) Scan parquet default.item +Output [2]: [i_category#12, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#16] +(15) ColumnarToRow [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] -(23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#16] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +(16) Filter [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] +Condition : (isnotnull(i_category#12) AND (i_category#12 = Electronics )) -(24) Project [codegen id : 3] -Output [1]: [i_manufact_id#16] -Input [2]: [i_category#15, i_manufact_id#16] +(17) Project [codegen id : 2] +Output [1]: [i_manufact_id#13] +Input [2]: [i_category#12, i_manufact_id#13] -(25) BroadcastExchange -Input [1]: [i_manufact_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +(18) BroadcastExchange +Input [1]: [i_manufact_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#16] +(19) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_manufact_id#11] +Right keys [1]: [i_manufact_id#13] Join condition: None +(20) BroadcastExchange +Input [2]: [i_item_sk#10, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#10] +Join condition: None + +(22) Project [codegen id : 5] +Output [3]: [ss_addr_sk#2, ss_ext_sales_price#3, i_manufact_id#11] +Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, i_item_sk#10, i_manufact_id#11] + +(23) Scan parquet default.customer_address +Output [2]: [ca_address_sk#16, ca_gmt_offset#17] +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#16, ca_gmt_offset#17] + +(25) Filter [codegen id : 4] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] +Condition : ((isnotnull(ca_gmt_offset#17) AND (ca_gmt_offset#17 = -5.00)) AND isnotnull(ca_address_sk#16)) + +(26) Project [codegen id : 4] +Output [1]: [ca_address_sk#16] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] + (27) BroadcastExchange -Input [2]: [i_item_sk#13, i_manufact_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#16] Join condition: None (29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_manufact_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#13, i_manufact_id#14] +Output [2]: [ss_ext_sales_price#3, i_manufact_id#11] +Input [4]: [ss_addr_sk#2, ss_ext_sales_price#3, i_manufact_id#11, ca_address_sk#16] (30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +Input [2]: [ss_ext_sales_price#3, i_manufact_id#11] +Keys [1]: [i_manufact_id#11] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#19] -Results [2]: [i_manufact_id#14, sum#20] +Results [2]: [i_manufact_id#11, sum#20] (31) Exchange -Input [2]: [i_manufact_id#14, sum#20] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [i_manufact_id#11, sum#20] +Arguments: hashpartitioning(i_manufact_id#11, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#14, sum#20] -Keys [1]: [i_manufact_id#14] +Input [2]: [i_manufact_id#11, sum#20] +Keys [1]: [i_manufact_id#11] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#22] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2) AS total_sales#23] +Results [2]: [i_manufact_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2) AS total_sales#23] (33) Scan parquet default.catalog_sales Output [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] @@ -244,47 +244,47 @@ Join condition: None Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] -(39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +(39) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#29, i_manufact_id#30] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#29] Join condition: None (41) Project [codegen id : 11] -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#29] +Output [3]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#30] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#29, i_manufact_id#30] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#30, i_manufact_id#31] +Output [1]: [ca_address_sk#31] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Left keys [1]: [cs_bill_addr_sk#24] +Right keys [1]: [ca_address_sk#31] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#30] +Input [4]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#30, ca_address_sk#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] -Keys [1]: [i_manufact_id#31] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#30] +Keys [1]: [i_manufact_id#30] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum#32] -Results [2]: [i_manufact_id#31, sum#33] +Results [2]: [i_manufact_id#30, sum#33] (46) Exchange -Input [2]: [i_manufact_id#31, sum#33] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_manufact_id#30, sum#33] +Arguments: hashpartitioning(i_manufact_id#30, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#33] -Keys [1]: [i_manufact_id#31] +Input [2]: [i_manufact_id#30, sum#33] +Keys [1]: [i_manufact_id#30] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Results [2]: [i_manufact_id#30, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] @@ -313,71 +313,71 @@ Join condition: None Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(54) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#42, i_manufact_id#43] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, i_manufact_id#43] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, i_item_sk#42, i_manufact_id#43] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_manufact_id#44] +Output [1]: [ca_address_sk#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#44] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#43] +Input [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, i_manufact_id#43, ca_address_sk#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] -Keys [1]: [i_manufact_id#44] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#43] +Keys [1]: [i_manufact_id#43] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] Aggregate Attributes [1]: [sum#45] -Results [2]: [i_manufact_id#44, sum#46] +Results [2]: [i_manufact_id#43, sum#46] (61) Exchange -Input [2]: [i_manufact_id#44, sum#46] -Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [i_manufact_id#43, sum#46] +Arguments: hashpartitioning(i_manufact_id#43, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#44, sum#46] -Keys [1]: [i_manufact_id#44] +Input [2]: [i_manufact_id#43, sum#46] +Keys [1]: [i_manufact_id#43] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +Results [2]: [i_manufact_id#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union (64) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#14, total_sales#23] -Keys [1]: [i_manufact_id#14] +Input [2]: [i_manufact_id#11, total_sales#23] +Keys [1]: [i_manufact_id#11] Functions [1]: [partial_sum(total_sales#23)] Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Results [3]: [i_manufact_id#11, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [3]: [i_manufact_id#11, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#11, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] -Keys [1]: [i_manufact_id#14] +Input [3]: [i_manufact_id#11, sum#52, isEmpty#53] +Keys [1]: [i_manufact_id#11] Functions [1]: [sum(total_sales#23)] Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] +Results [2]: [i_manufact_id#11, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] +Input [2]: [i_manufact_id#11, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#11, total_sales#56] ===== Subqueries ===== 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 d1a115fa528d5..1b29ee6b28a9c 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_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] + 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] Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_addr_sk,ss_item_sk] @@ -35,28 +35,28 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] + WholeStageCodegen (3) + 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 #5 + WholeStageCodegen (2) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + 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 #6 - WholeStageCodegen (3) - 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] WholeStageCodegen (12) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter @@ -64,9 +64,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_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + 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] Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_bill_addr_sk,cs_item_sk] @@ -77,9 +77,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [i_item_sk,i_manufact_id] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #6 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter @@ -87,9 +87,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_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + 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] Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_addr_sk,ws_item_sk] @@ -100,6 +100,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [ca_address_sk] #4 + ReusedExchange [i_item_sk,i_manufact_id] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #6 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 a737a06529ae8..aa943cd6a7473 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,128 +6,128 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * 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) + : +- * 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.item (11) + +- Scan parquet default.date_dim (11) -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +(1) Scan parquet default.store_sales +Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] 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: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(2) ColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(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)) +(3) Filter [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_item_sk#1) -(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] +(4) Scan parquet default.item +Output [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] +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 -(5) BroadcastExchange -Input [2]: [d_date_sk#1, d_year#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(5) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] -(6) Scan parquet default.store_sales -Output [3]: [ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] +Condition : ((isnotnull(i_manager_id#8) AND (i_manager_id#8 = 1)) AND isnotnull(i_item_sk#5)) -(7) ColumnarToRow -Input [3]: [ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] +(7) Project [codegen id : 1] +Output [3]: [i_item_sk#5, i_brand_id#6, i_brand#7] +Input [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] -(8) Filter -Input [3]: [ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_item_sk#5) +(8) BroadcastExchange +Input [3]: [i_item_sk#5, i_brand_id#6, i_brand#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#7] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6] -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] +Output [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_brand_id#6, i_brand#7] +Input [6]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_brand_id#6, i_brand#7] -(11) Scan parquet default.item -Output [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] 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 +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 (12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] (13) Filter [codegen id : 2] -Input [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] -Condition : ((isnotnull(i_manager_id#12) AND (i_manager_id#12 = 1)) AND isnotnull(i_item_sk#9)) +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 2000)) AND isnotnull(d_date_sk#10)) (14) Project [codegen id : 2] -Output [3]: [i_item_sk#9, i_brand_id#10, i_brand#11] -Input [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] +Output [2]: [d_date_sk#10, d_year#11] +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] (15) BroadcastExchange -Input [3]: [i_item_sk#9, i_brand_id#10, i_brand#11] +Input [2]: [d_date_sk#10, d_year#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#10] Join condition: None (17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#6, i_brand_id#10, i_brand#11] -Input [6]: [d_year#2, ss_item_sk#5, ss_ext_sales_price#6, i_item_sk#9, i_brand_id#10, i_brand#11] +Output [4]: [d_year#11, ss_ext_sales_price#2, i_brand_id#6, i_brand#7] +Input [6]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_brand_id#6, i_brand#7, d_date_sk#10, d_year#11] (18) HashAggregate [codegen id : 3] -Input [4]: [d_year#2, ss_ext_sales_price#6, i_brand_id#10, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] +Input [4]: [d_year#11, ss_ext_sales_price#2, i_brand_id#6, i_brand#7] +Keys [3]: [d_year#11, i_brand#7, i_brand_id#6] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#14] -Results [4]: [d_year#2, i_brand#11, i_brand_id#10, sum#15] +Results [4]: [d_year#11, i_brand#7, i_brand_id#6, sum#15] (19) Exchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#10, sum#15] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#10, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [4]: [d_year#11, i_brand#7, i_brand_id#6, sum#15] +Arguments: hashpartitioning(d_year#11, i_brand#7, i_brand_id#6, 5), ENSURE_REQUIREMENTS, [id=#16] (20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#11, i_brand_id#10, sum#15] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#17] -Results [4]: [d_year#2, i_brand_id#10 AS brand_id#18, i_brand#11 AS brand#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#17,17,2) AS ext_price#20] +Input [4]: [d_year#11, i_brand#7, i_brand_id#6, sum#15] +Keys [3]: [d_year#11, i_brand#7, i_brand_id#6] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] +Results [4]: [d_year#11, i_brand_id#6 AS brand_id#18, i_brand#7 AS brand#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS ext_price#20] (21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#18, brand#19, ext_price#20] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [d_year#2, brand_id#18, brand#19, ext_price#20] +Input [4]: [d_year#11, brand_id#18, brand#19, ext_price#20] +Arguments: 100, [d_year#11 ASC NULLS FIRST, ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [d_year#11, brand_id#18, brand#19, ext_price#20] ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 ReusedExchange (22) -(22) ReusedExchange [Reuses operator id: 5] -Output [2]: [d_date_sk#1, d_year#2] +(22) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#10, d_year#11] 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 6caf3262b7b8f..c00005c8eb798 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,28 +6,28 @@ 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_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 [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (2) - 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] 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 1fa6b7f39bc1f..c8d2a4748ba95 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,116 +6,116 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * 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) + : +- * 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.item (11) + +- Scan parquet default.date_dim (11) -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +(1) Scan parquet default.store_sales +Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] 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: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(2) ColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(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)) +(3) Filter [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_item_sk#1) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(4) Scan parquet default.item +Output [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] +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 -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(5) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] -(6) Scan parquet default.store_sales -Output [3]: [ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] +Condition : ((isnotnull(i_manager_id#8) AND (i_manager_id#8 = 28)) AND isnotnull(i_item_sk#5)) -(7) ColumnarToRow -Input [3]: [ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] +(7) Project [codegen id : 1] +Output [3]: [i_item_sk#5, i_brand_id#6, i_brand#7] +Input [4]: [i_item_sk#5, i_brand_id#6, i_brand#7, i_manager_id#8] -(8) Filter -Input [3]: [ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_item_sk#5) +(8) BroadcastExchange +Input [3]: [i_item_sk#5, i_brand_id#6, i_brand#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#7] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [2]: [ss_item_sk#5, ss_ext_sales_price#6] -Input [4]: [d_date_sk#1, ss_item_sk#5, ss_ext_sales_price#6, ss_sold_date_sk#7] +Output [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_brand_id#6, i_brand#7] +Input [6]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_brand_id#6, i_brand#7] -(11) Scan parquet default.item -Output [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] 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 +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 (12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] (13) Filter [codegen id : 2] -Input [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] -Condition : ((isnotnull(i_manager_id#12) AND (i_manager_id#12 = 28)) AND isnotnull(i_item_sk#9)) +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) (14) Project [codegen id : 2] -Output [3]: [i_item_sk#9, i_brand_id#10, i_brand#11] -Input [4]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manager_id#12] +Output [1]: [d_date_sk#10] +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] (15) BroadcastExchange -Input [3]: [i_item_sk#9, i_brand_id#10, i_brand#11] +Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#10] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#6, i_brand_id#10, i_brand#11] -Input [5]: [ss_item_sk#5, ss_ext_sales_price#6, i_item_sk#9, i_brand_id#10, i_brand#11] +Output [3]: [ss_ext_sales_price#2, i_brand_id#6, i_brand#7] +Input [5]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_brand_id#6, i_brand#7, d_date_sk#10] (18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#6, i_brand_id#10, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] +Input [3]: [ss_ext_sales_price#2, i_brand_id#6, i_brand#7] +Keys [2]: [i_brand#7, i_brand_id#6] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#14] -Results [3]: [i_brand#11, i_brand_id#10, sum#15] +Results [3]: [i_brand#7, i_brand_id#6, sum#15] (19) Exchange -Input [3]: [i_brand#11, i_brand_id#10, sum#15] -Arguments: hashpartitioning(i_brand#11, i_brand_id#10, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [3]: [i_brand#7, i_brand_id#6, sum#15] +Arguments: hashpartitioning(i_brand#7, i_brand_id#6, 5), ENSURE_REQUIREMENTS, [id=#16] (20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#11, i_brand_id#10, sum#15] -Keys [2]: [i_brand#11, i_brand_id#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#17] -Results [3]: [i_brand_id#10 AS brand_id#18, i_brand#11 AS brand#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#17,17,2) AS ext_price#20] +Input [3]: [i_brand#7, i_brand_id#6, sum#15] +Keys [2]: [i_brand#7, i_brand_id#6] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] +Results [3]: [i_brand_id#6 AS brand_id#18, i_brand#7 AS brand#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS ext_price#20] (21) TakeOrderedAndProject Input [3]: [brand_id#18, brand#19, ext_price#20] @@ -123,11 +123,11 @@ Arguments: 100, [ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [br ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 ReusedExchange (22) -(22) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#1] +(22) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#10] 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 48a1308dfc427..4208d6ce88e97 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,28 +6,28 @@ 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_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 [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (2) - 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] 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 339dc5d95e4c4..95092a8bbcc51 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 (80) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * 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 (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 (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * 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) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -85,7 +85,7 @@ TakeOrderedAndProject (80) Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct @@ -97,213 +97,213 @@ Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_s Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) (4) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#9, hd_buy_potential#10] +Output [2]: [hd_demo_sk#10, hd_buy_potential#11] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Input [2]: [hd_demo_sk#10, hd_buy_potential#11] (6) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) +Input [2]: [hd_demo_sk#10, hd_buy_potential#11] +Condition : ((isnotnull(hd_buy_potential#11) AND (hd_buy_potential#11 = >10000 )) AND isnotnull(hd_demo_sk#10)) (7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#9] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Output [1]: [hd_demo_sk#10] +Input [2]: [hd_demo_sk#10, hd_buy_potential#11] (8) BroadcastExchange -Input [1]: [hd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Input [1]: [hd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (9) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#9] +Right keys [1]: [hd_demo_sk#10] Join condition: None (10) Project [codegen id : 4] Output [7]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#9] +Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#10] (11) Scan parquet default.customer_demographics -Output [2]: [cd_demo_sk#12, cd_marital_status#13] +Output [2]: [cd_demo_sk#13, cd_marital_status#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,D), IsNotNull(cd_demo_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [cd_demo_sk#12, cd_marital_status#13] +Input [2]: [cd_demo_sk#13, cd_marital_status#14] (13) Filter [codegen id : 2] -Input [2]: [cd_demo_sk#12, cd_marital_status#13] -Condition : ((isnotnull(cd_marital_status#13) AND (cd_marital_status#13 = D)) AND isnotnull(cd_demo_sk#12)) +Input [2]: [cd_demo_sk#13, cd_marital_status#14] +Condition : ((isnotnull(cd_marital_status#14) AND (cd_marital_status#14 = D)) AND isnotnull(cd_demo_sk#13)) (14) Project [codegen id : 2] -Output [1]: [cd_demo_sk#12] -Input [2]: [cd_demo_sk#12, cd_marital_status#13] +Output [1]: [cd_demo_sk#13] +Input [2]: [cd_demo_sk#13, cd_marital_status#14] (15) BroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [cd_demo_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#12] +Right keys [1]: [cd_demo_sk#13] Join condition: None (17) Project [codegen id : 4] Output [6]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#12] +Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#13] (18) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_date#16] +Output [2]: [d_date_sk#16, d_date#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#15, d_date#16] +Input [2]: [d_date_sk#16, d_date#17] (20) Filter [codegen id : 3] -Input [2]: [d_date_sk#15, d_date#16] -Condition : (isnotnull(d_date#16) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_date#17] +Condition : (isnotnull(d_date#17) AND isnotnull(d_date_sk#16)) (21) BroadcastExchange -Input [2]: [d_date_sk#15, d_date#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +Input [2]: [d_date_sk#16, d_date#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] (22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#16] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] -Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#15, d_date#16] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#16, d_date#17] (24) Exchange -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] +Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#19] (25) Sort [codegen id : 5] -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] Arguments: [cs_item_sk#4 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item -Output [2]: [i_item_sk#19, i_item_desc#20] +Output [2]: [i_item_sk#20, i_item_desc#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#19, i_item_desc#20] +Input [2]: [i_item_sk#20, i_item_desc#21] (28) Filter [codegen id : 6] -Input [2]: [i_item_sk#19, i_item_desc#20] -Condition : isnotnull(i_item_sk#19) +Input [2]: [i_item_sk#20, i_item_desc#21] +Condition : isnotnull(i_item_sk#20) (29) Exchange -Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [i_item_sk#20, i_item_desc#21] +Arguments: hashpartitioning(i_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#22] (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 +Input [2]: [i_item_sk#20, i_item_desc#21] +Arguments: [i_item_sk#20 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#19] +Right keys [1]: [i_item_sk#20] Join condition: None -(32) Project [codegen id : 8] -Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] -Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_sk#19, i_item_desc#20] +(32) Project [codegen id : 10] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21] +Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21] -(33) Exchange -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] -Arguments: hashpartitioning(cs_item_sk#4, cs_sold_date_sk#8, 5), ENSURE_REQUIREMENTS, [id=#22] - -(34) Sort [codegen id : 9] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] -Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_sold_date_sk#8 ASC NULLS FIRST], false, 0 - -(35) Scan parquet default.date_dim +(33) 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 -(36) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 8] Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) Filter [codegen id : 10] +(35) Filter [codegen id : 8] 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)) -(38) Project [codegen id : 10] +(36) Project [codegen id : 8] 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] -(39) BroadcastExchange +(37) 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] -(40) Scan parquet default.date_dim +(38) 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 -(41) ColumnarToRow +(39) ColumnarToRow Input [2]: [d_date_sk#28, d_week_seq#29] -(42) Filter +(40) Filter Input [2]: [d_date_sk#28, d_week_seq#29] Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(43) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#25] Right keys [1]: [d_week_seq#29] Join condition: None -(44) Project [codegen id : 11] +(42) Project [codegen id : 9] 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] -(45) BroadcastExchange +(43) 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] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#8] +Right keys [1]: [d_date_sk#23] +Join condition: (d_date#17 > d_date#24 + 5 days) + +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] + +(46) Exchange +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28] +Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] -(46) Scan parquet default.inventory -Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#28 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.inventory +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(47) ColumnarToRow -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] - -(48) Filter -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] -Condition : ((isnotnull(inv_quantity_on_hand#33) AND isnotnull(inv_item_sk#31)) AND isnotnull(inv_warehouse_sk#32)) - -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [d_date_sk#28] -Right keys [1]: [inv_date_sk#34] -Join condition: None +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] -(50) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +(50) Filter [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : ((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#36, w_warehouse_name#37] @@ -324,30 +324,30 @@ Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [inv_warehouse_sk#32] +Left keys [1]: [inv_warehouse_sk#33] Right keys [1]: [w_warehouse_sk#36] Join condition: None (56) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, w_warehouse_sk#36, w_warehouse_name#37] +Output [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_sk#36, w_warehouse_name#37] (57) Exchange -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] -Arguments: hashpartitioning(inv_item_sk#31, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#35, 5), ENSURE_REQUIREMENTS, [id=#39] (58) Sort [codegen id : 14] -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] -Arguments: [inv_item_sk#31 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#35 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#4, cs_sold_date_sk#8] -Right keys [2]: [inv_item_sk#31, d_date_sk#23] -Join condition: ((inv_quantity_on_hand#33 < cs_quantity#7) AND (d_date#16 > d_date#24 + 5 days)) +Left keys [2]: [cs_item_sk#4, d_date_sk#28] +Right keys [2]: [inv_item_sk#32, inv_date_sk#35] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#7) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Input [13]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28, inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#40] @@ -373,15 +373,15 @@ Right keys [1]: [p_promo_sk#40] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25, p_promo_sk#40] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25, p_promo_sk#40] (67) Exchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#42] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -416,38 +416,38 @@ Right keys [2]: [cr_item_sk#43, cr_order_number#44] Join condition: None (76) Project [codegen id : 20] -Output [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25, cr_item_sk#43, cr_order_number#44] +Output [3]: [w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25, cr_item_sk#43, cr_order_number#44] (77) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25] +Input [3]: [w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Keys [3]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#47] -Results [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count#48] +Results [4]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count#48] (78) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count#48] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#37, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count#48] +Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#37, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#49] (79) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count#48] -Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25] +Input [4]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count#48] +Keys [3]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#50] -Results [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count(1)#50 AS no_promo#51, count(1)#50 AS promo#52, count(1)#50 AS total_cnt#53] +Results [6]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count(1)#50 AS no_promo#51, count(1)#50 AS promo#52, count(1)#50 AS total_cnt#53] (80) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] -Arguments: 100, [total_cnt#53 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#37 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] +Input [6]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] +Arguments: 100, [total_cnt#53 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#37 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (81) -(81) ReusedExchange [Reuses operator id: 45] +(81) ReusedExchange [Reuses operator id: 43] Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] 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 5cc7212d36e82..c98d0fd9e1fc5 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,97 +16,97 @@ 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,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (11) + Sort [cs_item_sk,d_date_sk] InputAdapter - Exchange [cs_item_sk,cs_sold_date_sk] #3 - WholeStageCodegen (8) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,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_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow + 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_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,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_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastExchange #6 + 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] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #7 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - 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] + BroadcastExchange #8 + 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] #9 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] 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 #5 + 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] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,d_date_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_item_sk,d_date_sk] #9 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - 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] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - 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] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #10 + Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] 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 d2d70bef8c3ee..93cc597893dc2 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,357 +1,357 @@ == Physical Plan == TakeOrderedAndProject (61) +- * Project (60) - +- * 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_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] + +- * 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] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +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] (3) Filter [codegen id : 2] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) +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)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +(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] 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 +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#7] +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] (6) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#7] +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)) -(8) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(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] -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#7] Join condition: None -(10) Project [codegen id : 2] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] +(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] -(11) Exchange -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: hashpartitioning(cr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#9] +(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), ENSURE_REQUIREMENTS, [id=#20] -(12) Sort [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: [cr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +(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 -(13) Scan parquet default.customer_address -Output [2]: [ca_address_sk#10, ca_state#11] +(12) Scan parquet default.catalog_returns +Output [4]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] +PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#10, ca_state#11] +(13) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24] -(15) Filter [codegen id : 4] -Input [2]: [ca_address_sk#10, ca_state#11] -Condition : (isnotnull(ca_address_sk#10) AND isnotnull(ca_state#11)) +(14) Filter [codegen id : 5] +Input [4]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24] +Condition : (isnotnull(cr_returning_addr_sk#22) AND isnotnull(cr_returning_customer_sk#21)) -(16) Exchange -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: hashpartitioning(ca_address_sk#10, 5), ENSURE_REQUIREMENTS, [id=#12] +(15) Scan parquet default.date_dim +Output [2]: [d_date_sk#26, d_year#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) Sort [codegen id : 5] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 +(16) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#26, d_year#27] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#10] -Join condition: None +(17) Filter [codegen id : 4] +Input [2]: [d_date_sk#26, d_year#27] +Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_sk#26)) -(19) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#10, ca_state#11] - -(20) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#13] -Results [3]: [cr_returning_customer_sk#1, ca_state#11, sum#14] - -(21) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#14] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#15] - -(22) HashAggregate [codegen id : 15] -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#14] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#16] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#17, ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#16,17,2) AS ctr_total_return#19] - -(23) Filter [codegen id : 15] -Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] -Condition : isnotnull(ctr_total_return#19) - -(24) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(cr_returned_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct +(18) Project [codegen id : 4] +Output [1]: [d_date_sk#26] +Input [2]: [d_date_sk#26, d_year#27] -(25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +(19) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] -(26) Filter [codegen id : 8] -Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_returning_addr_sk#21) +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#26] +Join condition: None -(27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#24] +(21) Project [codegen id : 5] +Output [3]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23] +Input [5]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24, d_date_sk#26] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join condition: None +(22) Exchange +Input [3]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23] +Arguments: hashpartitioning(cr_returning_addr_sk#22, 5), ENSURE_REQUIREMENTS, [id=#29] -(29) Project [codegen id : 8] -Output [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23, d_date_sk#24] +(23) Sort [codegen id : 6] +Input [3]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23] +Arguments: [cr_returning_addr_sk#22 ASC NULLS FIRST], false, 0 -(30) Exchange -Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Arguments: hashpartitioning(cr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#30, ca_state#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct -(31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Arguments: [cr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#30, ca_state#31] -(32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#26, ca_state#27] +(26) Filter [codegen id : 7] +Input [2]: [ca_address_sk#30, ca_state#31] +Condition : (isnotnull(ca_address_sk#30) AND isnotnull(ca_state#31)) -(33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#26, ca_state#27] -Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 +(27) Exchange +Input [2]: [ca_address_sk#30, ca_state#31] +Arguments: hashpartitioning(ca_address_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] -(34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#21] -Right keys [1]: [ca_address_sk#26] -Join condition: None +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#30, ca_state#31] +Arguments: [ca_address_sk#30 ASC NULLS FIRST], false, 0 -(35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] -Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, ca_address_sk#26, ca_state#27] - -(36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] -Keys [2]: [cr_returning_customer_sk#20, ca_state#27] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#22))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] - -(37) Exchange -Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] -Arguments: hashpartitioning(cr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] - -(38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] -Keys [2]: [cr_returning_customer_sk#20, ca_state#27] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))#31] -Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#22))#31,17,2) AS ctr_total_return#19] - -(39) HashAggregate [codegen id : 13] -Input [2]: [ctr_state#18, ctr_total_return#19] -Keys [1]: [ctr_state#18] -Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ctr_state#18, sum#34, count#35] - -(40) Exchange -Input [3]: [ctr_state#18, sum#34, count#35] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] - -(41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#34, count#35] -Keys [1]: [ctr_state#18] -Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] - -(42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) - -(43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] - -(44) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#39] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) - -(45) Project [codegen id : 15] -Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] - -(46) Exchange -Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#41] - -(47) Sort [codegen id : 16] -Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: [ctr_customer_sk#17 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.customer -Output [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [cr_returning_addr_sk#22] +Right keys [1]: [ca_address_sk#30] +Join condition: None -(49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +(30) Project [codegen id : 9] +Output [3]: [cr_returning_customer_sk#21, cr_return_amt_inc_tax#23, ca_state#31] +Input [5]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, ca_address_sk#30, ca_state#31] + +(31) HashAggregate [codegen id : 9] +Input [3]: [cr_returning_customer_sk#21, cr_return_amt_inc_tax#23, ca_state#31] +Keys [2]: [cr_returning_customer_sk#21, ca_state#31] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#23))] +Aggregate Attributes [1]: [sum#33] +Results [3]: [cr_returning_customer_sk#21, ca_state#31, sum#34] + +(32) Exchange +Input [3]: [cr_returning_customer_sk#21, ca_state#31, sum#34] +Arguments: hashpartitioning(cr_returning_customer_sk#21, ca_state#31, 5), ENSURE_REQUIREMENTS, [id=#35] + +(33) HashAggregate [codegen id : 10] +Input [3]: [cr_returning_customer_sk#21, ca_state#31, sum#34] +Keys [2]: [cr_returning_customer_sk#21, ca_state#31] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#23))#36] +Results [3]: [cr_returning_customer_sk#21 AS ctr_customer_sk#37, ca_state#31 AS ctr_state#38, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#23))#36,17,2) AS ctr_total_return#39] + +(34) Filter [codegen id : 10] +Input [3]: [ctr_customer_sk#37, ctr_state#38, ctr_total_return#39] +Condition : isnotnull(ctr_total_return#39) + +(35) Exchange +Input [3]: [ctr_customer_sk#37, ctr_state#38, ctr_total_return#39] +Arguments: hashpartitioning(ctr_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#40] + +(36) Sort [codegen id : 11] +Input [3]: [ctr_customer_sk#37, ctr_state#38, ctr_total_return#39] +Arguments: [ctr_customer_sk#37 ASC NULLS FIRST], false, 0 + +(37) SortMergeJoin [codegen id : 20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ctr_customer_sk#37] +Join condition: None -(50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] -Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_current_addr_sk#44)) +(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#38, ctr_total_return#39] +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#37, ctr_state#38, ctr_total_return#39] -(51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +(39) Scan parquet default.catalog_returns +Output [4]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43, cr_returned_date_sk#44] 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: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#44), dynamicpruningexpression(cr_returned_date_sk#44 IN dynamicpruning#25)] +PushedFilters: [IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +(40) ColumnarToRow [codegen id : 13] +Input [4]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43, cr_returned_date_sk#44] -(53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#48)) +(41) Filter [codegen id : 13] +Input [4]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43, cr_returned_date_sk#44] +Condition : isnotnull(cr_returning_addr_sk#42) -(54) BroadcastExchange -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] +(42) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#45] -(55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#44] -Right keys [1]: [ca_address_sk#48] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None -(56) Project [codegen id : 18] -Output [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Input [18]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47, ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +(44) Project [codegen id : 13] +Output [3]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43] +Input [5]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43, cr_returned_date_sk#44, d_date_sk#45] -(57) Exchange -Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: hashpartitioning(c_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#61] +(45) Exchange +Input [3]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43] +Arguments: hashpartitioning(cr_returning_addr_sk#42, 5), ENSURE_REQUIREMENTS, [id=#46] -(58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 14] +Input [3]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43] +Arguments: [cr_returning_addr_sk#42 ASC NULLS FIRST], false, 0 -(59) SortMergeJoin [codegen id : 20] -Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [c_customer_sk#42] +(47) ReusedExchange [Reuses operator id: 27] +Output [2]: [ca_address_sk#47, ca_state#48] + +(48) Sort [codegen id : 16] +Input [2]: [ca_address_sk#47, ca_state#48] +Arguments: [ca_address_sk#47 ASC NULLS FIRST], false, 0 + +(49) SortMergeJoin [codegen id : 17] +Left keys [1]: [cr_returning_addr_sk#42] +Right keys [1]: [ca_address_sk#47] Join condition: None +(50) Project [codegen id : 17] +Output [3]: [cr_returning_customer_sk#41, cr_return_amt_inc_tax#43, ca_state#48] +Input [5]: [cr_returning_customer_sk#41, cr_returning_addr_sk#42, cr_return_amt_inc_tax#43, ca_address_sk#47, ca_state#48] + +(51) HashAggregate [codegen id : 17] +Input [3]: [cr_returning_customer_sk#41, cr_return_amt_inc_tax#43, ca_state#48] +Keys [2]: [cr_returning_customer_sk#41, ca_state#48] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#43))] +Aggregate Attributes [1]: [sum#49] +Results [3]: [cr_returning_customer_sk#41, ca_state#48, sum#50] + +(52) Exchange +Input [3]: [cr_returning_customer_sk#41, ca_state#48, sum#50] +Arguments: hashpartitioning(cr_returning_customer_sk#41, ca_state#48, 5), ENSURE_REQUIREMENTS, [id=#51] + +(53) HashAggregate [codegen id : 18] +Input [3]: [cr_returning_customer_sk#41, ca_state#48, sum#50] +Keys [2]: [cr_returning_customer_sk#41, ca_state#48] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#43))#52] +Results [2]: [ca_state#48 AS ctr_state#38, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#43))#52,17,2) AS ctr_total_return#39] + +(54) HashAggregate [codegen id : 18] +Input [2]: [ctr_state#38, ctr_total_return#39] +Keys [1]: [ctr_state#38] +Functions [1]: [partial_avg(ctr_total_return#39)] +Aggregate Attributes [2]: [sum#53, count#54] +Results [3]: [ctr_state#38, sum#55, count#56] + +(55) Exchange +Input [3]: [ctr_state#38, sum#55, count#56] +Arguments: hashpartitioning(ctr_state#38, 5), ENSURE_REQUIREMENTS, [id=#57] + +(56) HashAggregate [codegen id : 19] +Input [3]: [ctr_state#38, sum#55, count#56] +Keys [1]: [ctr_state#38] +Functions [1]: [avg(ctr_total_return#39)] +Aggregate Attributes [1]: [avg(ctr_total_return#39)#58] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#39)#58) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#59, ctr_state#38 AS ctr_state#38#60] + +(57) Filter [codegen id : 19] +Input [2]: [(avg(ctr_total_return) * 1.2)#59, ctr_state#38#60] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#59) + +(58) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#59, ctr_state#38#60] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#61] + +(59) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [ctr_state#38] +Right keys [1]: [ctr_state#38#60] +Join condition: (cast(ctr_total_return#39 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#59) + (60) Project [codegen id : 20] -Output [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] -Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +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#39] +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#38, ctr_total_return#39, (avg(ctr_total_return) * 1.2)#59, ctr_state#38#60] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] -Arguments: 100, [c_customer_id#43 ASC NULLS FIRST, c_salutation#45 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, c_last_name#47 ASC NULLS FIRST, ca_street_number#49 ASC NULLS FIRST, ca_street_name#50 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#53 ASC NULLS FIRST, ca_county#54 ASC NULLS FIRST, ca_state#55 ASC NULLS FIRST, ca_zip#56 ASC NULLS FIRST, ca_country#57 ASC NULLS FIRST, ca_gmt_offset#58 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +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#39] +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#39 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#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:1 Hosting operator id = 12 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 ReusedExchange (62) -(62) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(62) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#26] -Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#23 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 39 Hosting Expression = cr_returned_date_sk#44 IN dynamicpruning#25 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 ce7d5b9627a4b..cee20aeec3948 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,29 +1,48 @@ 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] - 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,(avg(ctr_total_return) * 1.2)] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + 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) 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] #2 - WholeStageCodegen (6) + Exchange [cr_returning_customer_sk,ca_state] #4 + WholeStageCodegen (9) 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 (3) + WholeStageCodegen (6) Sort [cr_returning_addr_sk] InputAdapter - Exchange [cr_returning_addr_sk] #3 - WholeStageCodegen (2) + Exchange [cr_returning_addr_sk] #5 + WholeStageCodegen (5) Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returning_addr_sk,cr_returning_customer_sk] @@ -31,77 +50,58 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st InputAdapter Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #6 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) Project [d_date_sk] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (8) Sort [ca_address_sk] InputAdapter - Exchange [ca_address_sk] #5 - WholeStageCodegen (4) + Exchange [ca_address_sk] #7 + WholeStageCodegen (7) Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),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_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - WholeStageCodegen (11) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #5 InputAdapter - 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 + BroadcastExchange #8 + WholeStageCodegen (19) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),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] 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 #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] + 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_returning_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + WholeStageCodegen (16) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #7 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 3554ce5dcb117..ef2ddc2466f18 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,207 +8,207 @@ +- * BroadcastHashJoin Inner BuildRight (41) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * 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) + : :- * 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) : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_demographics (31) + : +- * 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) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(1) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3, cr_returned_date_sk#4] +(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] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +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 -(2) ColumnarToRow [codegen id : 2] -Input [4]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3, cr_returned_date_sk#4] +(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] -(3) Filter [codegen id : 2] -Input [4]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_call_center_sk#2) AND isnotnull(cr_returning_customer_sk#1)) +(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)) -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] 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 +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 (5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] (6) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 11)) AND isnotnull(d_date_sk#6)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#6] -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +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)) -(8) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] +(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] -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +(8) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#5] Join condition: None -(10) Project [codegen id : 2] -Output [3]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3] -Input [5]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3, cr_returned_date_sk#4, d_date_sk#6] - -(11) BroadcastExchange -Input [3]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +(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] -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(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}/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}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct + +(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)) -(13) ColumnarToRow -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(13) Project [codegen id : 2] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(14) Filter -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Condition : (((isnotnull(c_customer_sk#11) AND isnotnull(c_current_addr_sk#14)) AND isnotnull(c_current_cdemo_sk#12)) AND isnotnull(c_current_hdemo_sk#13)) +(14) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#1] -Right keys [1]: [c_customer_sk#11] +Left keys [1]: [c_current_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#9] Join condition: None (16) Project [codegen id : 7] -Output [5]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Input [7]: [cr_returning_customer_sk#1, cr_call_center_sk#2, cr_net_loss#3, c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +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] -(17) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#15, hd_buy_potential#16] +(17) Scan parquet default.customer_address +Output [2]: [ca_address_sk#12, ca_gmt_offset#13] 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 +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 (18) ColumnarToRow [codegen id : 3] -Input [2]: [hd_demo_sk#15, hd_buy_potential#16] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (19) Filter [codegen id : 3] -Input [2]: [hd_demo_sk#15, hd_buy_potential#16] -Condition : ((isnotnull(hd_buy_potential#16) AND StartsWith(hd_buy_potential#16, Unknown)) AND isnotnull(hd_demo_sk#15)) +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)) (20) Project [codegen id : 3] -Output [1]: [hd_demo_sk#15] -Input [2]: [hd_demo_sk#15, hd_buy_potential#16] +Output [1]: [ca_address_sk#12] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (21) BroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [ca_address_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#13] -Right keys [1]: [hd_demo_sk#15] +Left keys [1]: [c_current_addr_sk#4] +Right keys [1]: [ca_address_sk#12] Join condition: None (23) Project [codegen id : 7] -Output [4]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current_addr_sk#14] -Input [6]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14, hd_demo_sk#15] +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] -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17, cr_returned_date_sk#18] 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: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] +(25) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17, cr_returned_date_sk#18] -(26) Filter [codegen id : 4] -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) +(26) Filter [codegen id : 5] +Input [4]: [cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17, cr_returned_date_sk#18] +Condition : (isnotnull(cr_call_center_sk#16) AND isnotnull(cr_returning_customer_sk#15)) -(27) Project [codegen id : 4] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] +(27) Scan parquet default.date_dim +Output [3]: [d_date_sk#20, d_year#21, d_moy#22] +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 -(28) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +(28) ColumnarToRow [codegen id : 4] +Input [3]: [d_date_sk#20, d_year#21, d_moy#22] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#14] -Right keys [1]: [ca_address_sk#18] -Join condition: None +(29) Filter [codegen id : 4] +Input [3]: [d_date_sk#20, d_year#21, d_moy#22] +Condition : ((((isnotnull(d_year#21) AND isnotnull(d_moy#22)) AND (d_year#21 = 1998)) AND (d_moy#22 = 11)) AND isnotnull(d_date_sk#20)) -(30) Project [codegen id : 7] -Output [3]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12] -Input [5]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current_addr_sk#14, ca_address_sk#18] +(30) Project [codegen id : 4] +Output [1]: [d_date_sk#20] +Input [3]: [d_date_sk#20, d_year#21, d_moy#22] -(31) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -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 +(31) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(32) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] +(32) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join condition: None -(33) Filter [codegen id : 5] -Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown )) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree ))) AND isnotnull(cd_demo_sk#21)) +(33) Project [codegen id : 5] +Output [3]: [cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17] +Input [5]: [cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17, cr_returned_date_sk#18, d_date_sk#20] (34) BroadcastExchange -Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] +Input [3]: [cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#12] -Right keys [1]: [cd_demo_sk#21] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cr_returning_customer_sk#15] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cr_call_center_sk#2, cr_net_loss#3, cd_marital_status#22, cd_education_status#23] -Input [6]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] +Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#16, cr_net_loss#17] +Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#15, cr_call_center_sk#16, cr_net_loss#17] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#25, cc_call_center_id#26, cc_name#27, cc_manager#28] @@ -229,31 +229,31 @@ Input [4]: [cc_call_center_sk#25, cc_call_center_id#26, cc_name#27, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#2] +Left keys [1]: [cr_call_center_sk#16] Right keys [1]: [cc_call_center_sk#25] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cr_net_loss#3, cd_marital_status#22, cd_education_status#23] -Input [8]: [cr_call_center_sk#2, cr_net_loss#3, cd_marital_status#22, cd_education_status#23, cc_call_center_sk#25, cc_call_center_id#26, cc_name#27, cc_manager#28] +Output [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cr_net_loss#17, cd_marital_status#6, cd_education_status#7] +Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#16, cr_net_loss#17, cc_call_center_sk#25, cc_call_center_id#26, cc_name#27, cc_manager#28] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cr_net_loss#3, cd_marital_status#22, cd_education_status#23] -Keys [5]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#22, cd_education_status#23] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#3))] +Input [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cr_net_loss#17, cd_marital_status#6, cd_education_status#7] +Keys [5]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#6, cd_education_status#7] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#17))] Aggregate Attributes [1]: [sum#30] -Results [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#22, cd_education_status#23, sum#31] +Results [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#6, cd_education_status#7, sum#31] (44) Exchange -Input [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#22, cd_education_status#23, sum#31] -Arguments: hashpartitioning(cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#22, cd_education_status#23, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#6, cd_education_status#7, sum#31] +Arguments: hashpartitioning(cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#6, cd_education_status#7, 5), ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#22, cd_education_status#23, sum#31] -Keys [5]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#22, cd_education_status#23] -Functions [1]: [sum(UnscaledValue(cr_net_loss#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#3))#33] -Results [4]: [cc_call_center_id#26 AS Call_Center#34, cc_name#27 AS Call_Center_Name#35, cc_manager#28 AS Manager#36, MakeDecimal(sum(UnscaledValue(cr_net_loss#3))#33,17,2) AS Returns_Loss#37] +Input [6]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#6, cd_education_status#7, sum#31] +Keys [5]: [cc_call_center_id#26, cc_name#27, cc_manager#28, cd_marital_status#6, cd_education_status#7] +Functions [1]: [sum(UnscaledValue(cr_net_loss#17))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#17))#33] +Results [4]: [cc_call_center_id#26 AS Call_Center#34, cc_name#27 AS Call_Center_Name#35, cc_manager#28 AS Manager#36, MakeDecimal(sum(UnscaledValue(cr_net_loss#17))#33,17,2) AS Returns_Loss#37] (46) Exchange Input [4]: [Call_Center#34, Call_Center_Name#35, Manager#36, Returns_Loss#37] @@ -265,11 +265,11 @@ Arguments: [Returns_Loss#37 DESC NULLS LAST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:1 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#19 ReusedExchange (48) -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(48) ReusedExchange [Reuses operator id: 31] +Output [1]: [d_date_sk#20] 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 87abdfb2b2c25..03ef76bc63897 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,60 +10,60 @@ 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 [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] + 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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_hdemo_sk,hd_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_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - 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] + 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] 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 #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) 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 #6 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) 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 #7 + BroadcastExchange #6 WholeStageCodegen (5) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow + 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_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #7 InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + 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] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 98c1bc5671cd9..91f246ce2aa12 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -10,24 +10,24 @@ +- 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.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.store_sales @@ -38,128 +38,128 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#5] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] +(5) Sort [codegen id : 2] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Condition : (i_category#11 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#6)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#12] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#8] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(13) Scan parquet default.item -Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_date#14] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] +Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 1999-02-22)) AND (d_date#14 <= 1999-03-24)) AND isnotnull(d_date_sk#13)) -(16) Exchange -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#15] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#13] +Input [2]: [d_date_sk#13, d_date#14] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#13] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Input [8]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +Output [6]: [ss_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11, d_date_sk#13] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [ss_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#16] -Results [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] +Results [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] (21) Exchange -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Arguments: hashpartitioning(i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Arguments: hashpartitioning(i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, 5), ENSURE_REQUIREMENTS, [id=#18] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] -Results [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w1#22, i_item_id#10] +Results [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w1#22, i_item_id#7] (23) Exchange -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: hashpartitioning(i_class#13, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: hashpartitioning(i_class#10, 5), ENSURE_REQUIREMENTS, [id=#23] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: [i_class#13 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: [i_class#10 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10] -Arguments: [sum(_w1#22) windowspecdefinition(i_class#13, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#13] +Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7] +Arguments: [sum(_w1#22) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#10] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25, i_item_id#10] -Input [9]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, i_item_id#10, _we0#24] +Output [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25, i_item_id#7] +Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, i_item_id#7, _we0#24] (27) Exchange -Input [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25, i_item_id#10] -Arguments: rangepartitioning(i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25, i_item_id#7] +Arguments: rangepartitioning(i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#26] (28) Sort [codegen id : 10] -Input [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25, i_item_id#10] -Arguments: [i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25, i_item_id#7] +Arguments: [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], true, 0 (29) Project [codegen id : 10] -Output [6]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] -Input [7]: [i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25, i_item_id#10] +Output [6]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] +Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25, i_item_id#7] ===== Subqueries ===== @@ -167,7 +167,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dyn ReusedExchange (30) -(30) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(30) ReusedExchange [Reuses operator id: 17] +Output [1]: [d_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt index 872ab90b8c3c8..54ad1aead4c6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt @@ -18,36 +18,36 @@ WholeStageCodegen (10) WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (2) - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index dd85594160c8f..d4acaa2cec459 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.web_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.web_sales @@ -36,120 +36,120 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#5] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] +(5) Sort [codegen id : 2] +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Condition : (i_category#11 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#6)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#12] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#1, ws_ext_sales_price#2] -Input [4]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, d_date_sk#5] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#1, ws_ext_sales_price#2] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#8] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#1, ws_ext_sales_price#2] -Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(13) Scan parquet default.item -Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_date#14] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] +Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 1999-02-22)) AND (d_date#14 <= 1999-03-24)) AND isnotnull(d_date_sk#13)) -(16) Exchange -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#15] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#13] +Input [2]: [d_date_sk#13, d_date#14] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#9] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#13] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Input [8]: [ws_item_sk#1, ws_ext_sales_price#2, i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +Output [6]: [ws_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11, d_date_sk#13] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [ws_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum#16] -Results [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] +Results [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] (21) Exchange -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Arguments: hashpartitioning(i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Arguments: hashpartitioning(i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, 5), ENSURE_REQUIREMENTS, [id=#18] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#19] -Results [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w1#22] +Results [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#19,17,2) AS _w1#22] (23) Exchange -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: hashpartitioning(i_class#13, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: hashpartitioning(i_class#10, 5), ENSURE_REQUIREMENTS, [id=#23] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: [i_class#13 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: [i_class#10 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: [sum(_w1#22) windowspecdefinition(i_class#13, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#13] +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: [sum(_w1#22) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#10] (26) Project [codegen id : 9] -Output [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25] -Input [9]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, _we0#24] +Output [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25] +Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, _we0#24] (27) TakeOrderedAndProject -Input [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] -Arguments: 100, [i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] +Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] ===== Subqueries ===== @@ -157,7 +157,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dyn ReusedExchange (28) -(28) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(28) ReusedExchange [Reuses operator id: 17] +Output [1]: [d_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt index 1babdd704b9c8..801a70eda99ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt @@ -14,36 +14,36 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ws_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #4 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 56fa48006d894..35e3304de7082 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) - : +- * 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) + : +- * 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) :- * HashAggregate (76) : +- Exchange (75) : +- * HashAggregate (74) @@ -280,89 +280,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 : 6] +(27) ColumnarToRow [codegen id : 7] Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(28) Filter [codegen id : 6] +(28) Filter [codegen id : 7] Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) -(29) Project [codegen id : 6] +(29) Project [codegen id : 7] Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(30) Exchange -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#28] - -(31) Sort [codegen id : 7] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#29] +(30) Scan parquet default.customer_address +Output [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [1]: [cd_demo_sk#29] +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 -(34) Filter [codegen id : 8] -Input [1]: [cd_demo_sk#29] -Condition : isnotnull(cd_demo_sk#29) +(31) ColumnarToRow [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(35) Exchange -Input [1]: [cd_demo_sk#29] -Arguments: hashpartitioning(cd_demo_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] +(32) Filter [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Condition : (ca_state#30 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#28)) -(36) Sort [codegen id : 9] -Input [1]: [cd_demo_sk#29] -Arguments: [cd_demo_sk#29 ASC NULLS FIRST], false, 0 +(33) BroadcastExchange +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(37) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#29] +(34) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#28] Join condition: None -(38) Project [codegen id : 11] -Output [3]: [c_customer_sk#23, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, cd_demo_sk#29] +(35) Project [codegen id : 7] +Output [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [8]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] +(36) Exchange +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#33] + +(37) Sort [codegen id : 8] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 + +(38) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#34] 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 +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] +(39) ColumnarToRow [codegen id : 9] +Input [1]: [cd_demo_sk#34] -(41) Filter [codegen id : 10] -Input [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(42) BroadcastExchange -Input [4]: [ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#35] +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 + +(43) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#34] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] -Input [7]: [c_customer_sk#23, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_county#32, ca_state#33, ca_country#34] +Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31, cd_demo_sk#34] (45) Exchange -Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -371,26 +371,26 @@ Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -Output [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#32, ca_state#33, ca_country#34] +Output [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] (49) HashAggregate [codegen id : 13] -Input [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [4]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32] +Input [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [4]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] Aggregate Attributes [14]: [sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] -Results [18]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Results [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] (50) Exchange -Input [18]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, ca_state#33, ca_county#32, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Arguments: hashpartitioning(i_item_id#20, ca_country#31, ca_state#30, ca_county#29, 5), ENSURE_REQUIREMENTS, [id=#72] (51) HashAggregate [codegen id : 14] -Input [18]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] -Keys [4]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32] +Input [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Keys [4]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, avg(agg4#40)#76, avg(agg5#41)#77, avg(agg6#42)#78, avg(agg7#43)#79] -Results [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] +Results [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] (52) ReusedExchange [Reuses operator id: unknown] Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] @@ -452,7 +452,7 @@ Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id= Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 35] +(66) ReusedExchange [Reuses operator id: 41] Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] @@ -567,7 +567,7 @@ Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 35] +(92) ReusedExchange [Reuses operator id: 41] Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] @@ -875,8 +875,8 @@ Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#3 (159) Union (160) TakeOrderedAndProject -Input [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] -Arguments: 100, [ca_country#34 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, i_item_id#20 ASC NULLS FIRST], [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] +Input [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] +Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#29 ASC NULLS FIRST, i_item_id#20 ASC NULLS FIRST], [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] ===== Subqueries ===== 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 0f8ddc8151c40..13aa887abaa2d 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 @@ -56,37 +56,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] - 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) + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (8) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (7) + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - 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 + 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 - 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 (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] 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 @@ -132,7 +132,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] #8 + ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (42) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -179,7 +179,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] #8 + ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (50) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index a74ad3e4b8b77..d9b14fcfb5bf9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.catalog_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.catalog_sales @@ -36,120 +36,120 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#5] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] +(5) Sort [codegen id : 2] +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Condition : (i_category#11 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#6)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#12] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#1, cs_ext_sales_price#2] -Input [4]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, d_date_sk#5] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#1, cs_ext_sales_price#2] -Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#8] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#1, cs_ext_sales_price#2] -Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(13) Scan parquet default.item -Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_date#14] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] +Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 1999-02-22)) AND (d_date#14 <= 1999-03-24)) AND isnotnull(d_date_sk#13)) -(16) Exchange -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#15] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#13] +Input [2]: [d_date_sk#13, d_date#14] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#9] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#13] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Input [8]: [cs_item_sk#1, cs_ext_sales_price#2, i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +Output [6]: [cs_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11, d_date_sk#13] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [cs_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum#16] -Results [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] +Results [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] (21) Exchange -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Arguments: hashpartitioning(i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Arguments: hashpartitioning(i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, 5), ENSURE_REQUIREMENTS, [id=#18] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#19] -Results [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w1#22] +Results [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#19,17,2) AS _w1#22] (23) Exchange -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: hashpartitioning(i_class#13, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: hashpartitioning(i_class#10, 5), ENSURE_REQUIREMENTS, [id=#23] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: [i_class#13 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: [i_class#10 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: [sum(_w1#22) windowspecdefinition(i_class#13, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#13] +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: [sum(_w1#22) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#10] (26) Project [codegen id : 9] -Output [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25] -Input [9]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, _we0#24] +Output [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25] +Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, _we0#24] (27) TakeOrderedAndProject -Input [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] -Arguments: 100, [i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] +Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] ===== Subqueries ===== @@ -157,7 +157,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dyn ReusedExchange (28) -(28) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(28) ReusedExchange [Reuses operator id: 17] +Output [1]: [d_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt index 7c18d19aac916..69e0754aed6d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt @@ -14,36 +14,36 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (2) - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #4 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 fe8ceb415f571..949f9c51b3d22 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 (80) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * 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 (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 (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * 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) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -85,7 +85,7 @@ TakeOrderedAndProject (80) Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct @@ -97,213 +97,213 @@ Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_s Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) (4) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#9, hd_buy_potential#10] +Output [2]: [hd_demo_sk#10, hd_buy_potential#11] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Input [2]: [hd_demo_sk#10, hd_buy_potential#11] (6) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) +Input [2]: [hd_demo_sk#10, hd_buy_potential#11] +Condition : ((isnotnull(hd_buy_potential#11) AND (hd_buy_potential#11 = 1001-5000 )) AND isnotnull(hd_demo_sk#10)) (7) Project [codegen id : 1] -Output [1]: [hd_demo_sk#9] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +Output [1]: [hd_demo_sk#10] +Input [2]: [hd_demo_sk#10, hd_buy_potential#11] (8) BroadcastExchange -Input [1]: [hd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Input [1]: [hd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (9) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#9] +Right keys [1]: [hd_demo_sk#10] Join condition: None (10) Project [codegen id : 4] Output [7]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#9] +Input [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, hd_demo_sk#10] (11) Scan parquet default.customer_demographics -Output [2]: [cd_demo_sk#12, cd_marital_status#13] +Output [2]: [cd_demo_sk#13, cd_marital_status#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,M), IsNotNull(cd_demo_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [cd_demo_sk#12, cd_marital_status#13] +Input [2]: [cd_demo_sk#13, cd_marital_status#14] (13) Filter [codegen id : 2] -Input [2]: [cd_demo_sk#12, cd_marital_status#13] -Condition : ((isnotnull(cd_marital_status#13) AND (cd_marital_status#13 = M)) AND isnotnull(cd_demo_sk#12)) +Input [2]: [cd_demo_sk#13, cd_marital_status#14] +Condition : ((isnotnull(cd_marital_status#14) AND (cd_marital_status#14 = M)) AND isnotnull(cd_demo_sk#13)) (14) Project [codegen id : 2] -Output [1]: [cd_demo_sk#12] -Input [2]: [cd_demo_sk#12, cd_marital_status#13] +Output [1]: [cd_demo_sk#13] +Input [2]: [cd_demo_sk#13, cd_marital_status#14] (15) BroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [cd_demo_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (16) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#12] +Right keys [1]: [cd_demo_sk#13] Join condition: None (17) Project [codegen id : 4] Output [6]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#12] +Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, cd_demo_sk#13] (18) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_date#16] +Output [2]: [d_date_sk#16, d_date#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#15, d_date#16] +Input [2]: [d_date_sk#16, d_date#17] (20) Filter [codegen id : 3] -Input [2]: [d_date_sk#15, d_date#16] -Condition : (isnotnull(d_date#16) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_date#17] +Condition : (isnotnull(d_date#17) AND isnotnull(d_date_sk#16)) (21) BroadcastExchange -Input [2]: [d_date_sk#15, d_date#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +Input [2]: [d_date_sk#16, d_date#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#18] (22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#16] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] -Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#15, d_date#16] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date_sk#16, d_date#17] (24) Exchange -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] +Arguments: hashpartitioning(cs_item_sk#4, 5), ENSURE_REQUIREMENTS, [id=#19] (25) Sort [codegen id : 5] -Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16] +Input [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17] Arguments: [cs_item_sk#4 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item -Output [2]: [i_item_sk#19, i_item_desc#20] +Output [2]: [i_item_sk#20, i_item_desc#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#19, i_item_desc#20] +Input [2]: [i_item_sk#20, i_item_desc#21] (28) Filter [codegen id : 6] -Input [2]: [i_item_sk#19, i_item_desc#20] -Condition : isnotnull(i_item_sk#19) +Input [2]: [i_item_sk#20, i_item_desc#21] +Condition : isnotnull(i_item_sk#20) (29) Exchange -Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [i_item_sk#20, i_item_desc#21] +Arguments: hashpartitioning(i_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#22] (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 +Input [2]: [i_item_sk#20, i_item_desc#21] +Arguments: [i_item_sk#20 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#19] +Right keys [1]: [i_item_sk#20] Join condition: None -(32) Project [codegen id : 8] -Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] -Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_sk#19, i_item_desc#20] +(32) Project [codegen id : 10] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21] +Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21] -(33) Exchange -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] -Arguments: hashpartitioning(cs_item_sk#4, cs_sold_date_sk#8, 5), ENSURE_REQUIREMENTS, [id=#22] - -(34) Sort [codegen id : 9] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20] -Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_sold_date_sk#8 ASC NULLS FIRST], false, 0 - -(35) Scan parquet default.date_dim +(33) 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 -(36) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 8] Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) Filter [codegen id : 10] +(35) Filter [codegen id : 8] 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)) -(38) Project [codegen id : 10] +(36) Project [codegen id : 8] 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] -(39) BroadcastExchange +(37) 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] -(40) Scan parquet default.date_dim +(38) 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 -(41) ColumnarToRow +(39) ColumnarToRow Input [2]: [d_date_sk#28, d_week_seq#29] -(42) Filter +(40) Filter Input [2]: [d_date_sk#28, d_week_seq#29] Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(43) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [d_week_seq#25] Right keys [1]: [d_week_seq#29] Join condition: None -(44) Project [codegen id : 11] +(42) Project [codegen id : 9] 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] -(45) BroadcastExchange +(43) 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] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#8] +Right keys [1]: [d_date_sk#23] +Join condition: (d_date#17 > d_date#24 + 5 days) + +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] + +(46) Exchange +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28] +Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] -(46) Scan parquet default.inventory -Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#28 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.inventory +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(47) ColumnarToRow -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] - -(48) Filter -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] -Condition : ((isnotnull(inv_quantity_on_hand#33) AND isnotnull(inv_item_sk#31)) AND isnotnull(inv_warehouse_sk#32)) - -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [d_date_sk#28] -Right keys [1]: [inv_date_sk#34] -Join condition: None +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] -(50) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +(50) Filter [codegen id : 13] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : ((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#36, w_warehouse_name#37] @@ -324,30 +324,30 @@ Input [2]: [w_warehouse_sk#36, w_warehouse_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [inv_warehouse_sk#32] +Left keys [1]: [inv_warehouse_sk#33] Right keys [1]: [w_warehouse_sk#36] Join condition: None (56) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, w_warehouse_sk#36, w_warehouse_name#37] +Output [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_sk#36, w_warehouse_name#37] (57) Exchange -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] -Arguments: hashpartitioning(inv_item_sk#31, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#35, 5), ENSURE_REQUIREMENTS, [id=#39] (58) Sort [codegen id : 14] -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] -Arguments: [inv_item_sk#31 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#35 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#4, cs_sold_date_sk#8] -Right keys [2]: [inv_item_sk#31, d_date_sk#23] -Join condition: ((inv_quantity_on_hand#33 < cs_quantity#7) AND (d_date#16 > d_date#24 + 5 days)) +Left keys [2]: [cs_item_sk#4, d_date_sk#28] +Right keys [2]: [inv_item_sk#32, inv_date_sk#35] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#7) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Input [13]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#31, inv_quantity_on_hand#33, w_warehouse_name#37] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#28, inv_item_sk#32, inv_quantity_on_hand#34, inv_date_sk#35, w_warehouse_name#37] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#40] @@ -373,15 +373,15 @@ Right keys [1]: [p_promo_sk#40] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25, p_promo_sk#40] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25, p_promo_sk#40] (67) Exchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#42] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -416,38 +416,38 @@ Right keys [2]: [cr_item_sk#43, cr_order_number#44] Join condition: None (76) Project [codegen id : 20] -Output [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#20, d_week_seq#25, cr_item_sk#43, cr_order_number#44] +Output [3]: [w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#37, i_item_desc#21, d_week_seq#25, cr_item_sk#43, cr_order_number#44] (77) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#37, i_item_desc#20, d_week_seq#25] -Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25] +Input [3]: [w_warehouse_name#37, i_item_desc#21, d_week_seq#25] +Keys [3]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#47] -Results [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count#48] +Results [4]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count#48] (78) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count#48] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#37, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count#48] +Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#37, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#49] (79) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count#48] -Keys [3]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25] +Input [4]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count#48] +Keys [3]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#50] -Results [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, count(1)#50 AS no_promo#51, count(1)#50 AS promo#52, count(1)#50 AS total_cnt#53] +Results [6]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, count(1)#50 AS no_promo#51, count(1)#50 AS promo#52, count(1)#50 AS total_cnt#53] (80) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] -Arguments: 100, [total_cnt#53 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#37 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] +Input [6]: [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] +Arguments: 100, [total_cnt#53 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#37 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#37, d_week_seq#25, no_promo#51, promo#52, total_cnt#53] ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (81) -(81) ReusedExchange [Reuses operator id: 45] +(81) ReusedExchange [Reuses operator id: 43] Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] 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 5cc7212d36e82..c98d0fd9e1fc5 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,97 +16,97 @@ 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,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (11) + Sort [cs_item_sk,d_date_sk] InputAdapter - Exchange [cs_item_sk,cs_sold_date_sk] #3 - WholeStageCodegen (8) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,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_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow + 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_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,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_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastExchange #6 + 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] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #7 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - 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] + BroadcastExchange #8 + 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] #9 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] 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 #5 + 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] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,d_date_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_item_sk,d_date_sk] #9 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - 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] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - 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] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #10 + Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index 89bb3e5d551a8..33d4b7d49e1d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -9,24 +9,24 @@ +- 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.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.store_sales @@ -37,124 +37,124 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#5] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] +(5) Sort [codegen id : 2] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Condition : (i_category#11 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#6)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#12] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#8] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#6, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] -(13) Scan parquet default.item -Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#13, d_date#14] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#13, d_date#14] +Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 1999-02-22)) AND (d_date#14 <= 1999-03-24)) AND isnotnull(d_date_sk#13)) -(16) Exchange -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: hashpartitioning(i_item_sk#9, 5), ENSURE_REQUIREMENTS, [id=#15] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#13] +Input [2]: [d_date_sk#13, d_date#14] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Arguments: [i_item_sk#9 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#13] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Input [8]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] +Output [6]: [ss_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11, d_date_sk#13] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#2, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [ss_ext_sales_price#2, i_item_id#7, i_item_desc#8, i_current_price#9, i_class#10, i_category#11] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#16] -Results [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] +Results [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] (21) Exchange -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Arguments: hashpartitioning(i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Arguments: hashpartitioning(i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, 5), ENSURE_REQUIREMENTS, [id=#18] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, sum#17] -Keys [5]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12] +Input [6]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, sum#17] +Keys [5]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] -Results [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w1#22] +Results [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS itemrevenue#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w0#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS _w1#22] (23) Exchange -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: hashpartitioning(i_class#13, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: hashpartitioning(i_class#10, 5), ENSURE_REQUIREMENTS, [id=#23] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: [i_class#13 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: [i_class#10 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22] -Arguments: [sum(_w1#22) windowspecdefinition(i_class#13, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#13] +Input [8]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22] +Arguments: [sum(_w1#22) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#24], [i_class#10] (26) Project [codegen id : 9] -Output [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25] -Input [9]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, _w0#21, _w1#22, _we0#24] +Output [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#21) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#24)), DecimalType(38,17), true) AS revenueratio#25] +Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, _w0#21, _w1#22, _we0#24] (27) Exchange -Input [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] -Arguments: rangepartitioning(i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] +Arguments: rangepartitioning(i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#26] (28) Sort [codegen id : 10] -Input [7]: [i_item_id#10, i_item_desc#11, i_category#14, i_class#13, i_current_price#12, itemrevenue#20, revenueratio#25] -Arguments: [i_category#14 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST, i_item_desc#11 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#20, revenueratio#25] +Arguments: [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#25 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -162,7 +162,7 @@ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dyn ReusedExchange (29) -(29) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(29) ReusedExchange [Reuses operator id: 17] +Output [1]: [d_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt index 2a8e0a077a7d9..45e9e64099cb7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt @@ -17,36 +17,36 @@ WholeStageCodegen (10) WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (2) - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date]