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 800d237e861ce..289e1a08d0894 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 @@ -1860,8 +1860,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 2e1cf4a137e25..7b591766cb4b2 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 25da173c8ecde..cfdd41a08844c 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 @@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) +Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000 ) OR (hd_buy_potential#14 = Unknown ))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#13] @@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] (26) Exchange Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, [id=#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (27) HashAggregate [codegen id : 5] Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] @@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, c_customer_sk#24, c_sa (35) Exchange Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22] -Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29] +Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#29] (36) Sort [codegen id : 7] Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22] 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 b4dd8173664b6..f4dc8d6c5d6df 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 @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -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_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: hashpartitioning(ws_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +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#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] 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#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] 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 e9b94a6b4651c..2207b0fee23ce 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,34 +14,34 @@ 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_sold_date_sk,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 (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - 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] #4 + 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 #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-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 327e7db702faa..3dc7034581000 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 @@ -57,7 +57,7 @@ Condition : ((((((isnotnull(ss_store_sk#5) AND isnotnull(ss_addr_sk#4)) AND isno Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -65,7 +65,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College ))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )))) (7) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -74,7 +74,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13] @@ -101,100 +101,100 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#3] Right keys [1]: [hd_demo_sk#15] -Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) +Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) (15) Project [codegen id : 6] Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] -(16) Scan parquet default.customer_address -Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#18, d_year#19] 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#18, ca_state#19, ca_country#20] +Input [2]: [d_date_sk#18, d_year#19] (18) Filter [codegen id : 3] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] -Condition : (((isnotnull(ca_country#20) AND (ca_country#20 = United States)) AND isnotnull(ca_address_sk#18)) AND ((ca_state#19 IN (TX,OH) OR ca_state#19 IN (OR,NM,KY)) OR ca_state#19 IN (VA,TX,MS))) +Input [2]: [d_date_sk#18, d_year#19] +Condition : ((isnotnull(d_year#19) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) (19) Project [codegen id : 3] -Output [2]: [ca_address_sk#18, ca_state#19] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_year#19] (20) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#18] -Join condition: ((((ca_state#19 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#19 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#19 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#18] +Join condition: None (22) Project [codegen id : 6] -Output [5]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [9]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#18, ca_state#19] +Output [6]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [8]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, d_date_sk#18] -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.store +Output [1]: [s_store_sk#21] 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#22, d_year#23] +Input [1]: [s_store_sk#21] (25) Filter [codegen id : 4] -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(26) Project [codegen id : 4] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Input [1]: [s_store_sk#21] +Condition : isnotnull(s_store_sk#21) -(27) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(26) BroadcastExchange +Input [1]: [s_store_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#21] Join condition: None -(29) Project [codegen id : 6] -Output [4]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [6]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, d_date_sk#22] +(28) Project [codegen id : 6] +Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] -(30) Scan parquet default.store -Output [1]: [s_store_sk#25] +(29) Scan parquet default.customer_address +Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] 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#23, ca_state#24, ca_country#25] -(31) ColumnarToRow [codegen id : 5] -Input [1]: [s_store_sk#25] +(31) Filter [codegen id : 5] +Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +Condition : (((isnotnull(ca_country#25) AND (ca_country#25 = United States)) AND isnotnull(ca_address_sk#23)) AND ((ca_state#24 IN (TX,OH) OR ca_state#24 IN (OR,NM,KY)) OR ca_state#24 IN (VA,TX,MS))) -(32) Filter [codegen id : 5] -Input [1]: [s_store_sk#25] -Condition : isnotnull(s_store_sk#25) +(32) Project [codegen id : 5] +Output [2]: [ca_address_sk#23, ca_state#24] +Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] (33) BroadcastExchange -Input [1]: [s_store_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#25] -Join condition: None +Left keys [1]: [ss_addr_sk#4] +Right keys [1]: [ca_address_sk#23] +Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [5]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, s_store_sk#25] +Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index 45d6c8f3b0bae..b457788dbd0b2 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_sold_date_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + 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_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] Project [ss_sold_date_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,cd_marital_status,cd_education_status] @@ -35,23 +35,23 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [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 #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 12e95ba50cd0d..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,261 +34,272 @@ 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 -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (3) Filter [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = F)) AND (cd_education_status#12 = Unknown)) AND isnotnull(cd_demo_sk#10)) +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (11) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#16, d_year#17] (13) Filter [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 1998)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 1998)) AND isnotnull(d_date_sk#16)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#16] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_year#17] (15) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (17) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (18) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#19, i_item_id#20] (20) Filter [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (21) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (23) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_sk#18, i_item_id#19] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#19, i_item_id#20] (24) Exchange -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 5] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (26) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true 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] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(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] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +(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] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(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#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] - -(31) Sort [codegen id : 7] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#28] +(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#28] +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#28] -Condition : isnotnull(cd_demo_sk#28) +(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#28] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +(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#28] -Arguments: [cd_demo_sk#28 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#23] -Right keys [1]: [cd_demo_sk#28] +(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#22, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] +(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#30, ca_county#31, ca_state#32, ca_country#33] +(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 + +(39) ColumnarToRow [codegen id : 9] +Input [1]: [cd_demo_sk#34] -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(41) Filter [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#30)) +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(42) BroadcastExchange -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +(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#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +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#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] +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#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +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] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +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#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] +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#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Keys [5]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Functions [7]: [partial_avg(cast(cs_quantity#5 as decimal(12,2))), partial_avg(cast(cs_list_price#6 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#8 as decimal(12,2))), partial_avg(cast(cs_sales_price#7 as decimal(12,2))), partial_avg(cast(cs_net_profit#9 as decimal(12,2))), partial_avg(cast(c_birth_year#26 as decimal(12,2))), partial_avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +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] +Keys [5]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#27 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [14]: [sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Results [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] (51) Exchange -Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] +Arguments: hashpartitioning(i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, 5), ENSURE_REQUIREMENTS, [id=#70] (52) HashAggregate [codegen id : 14] -Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [5]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Functions [7]: [avg(cast(cs_quantity#5 as decimal(12,2))), avg(cast(cs_list_price#6 as decimal(12,2))), avg(cast(cs_coupon_amt#8 as decimal(12,2))), avg(cast(cs_sales_price#7 as decimal(12,2))), avg(cast(cs_net_profit#9 as decimal(12,2))), avg(cast(c_birth_year#26 as decimal(12,2))), avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#5 as decimal(12,2)))#70, avg(cast(cs_list_price#6 as decimal(12,2)))#71, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#72, avg(cast(cs_sales_price#7 as decimal(12,2)))#73, avg(cast(cs_net_profit#9 as decimal(12,2)))#74, avg(cast(c_birth_year#26 as decimal(12,2)))#75, avg(cast(cd_dep_count#13 as decimal(12,2)))#76] -Results [11]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, avg(cast(cs_quantity#5 as decimal(12,2)))#70 AS agg1#77, avg(cast(cs_list_price#6 as decimal(12,2)))#71 AS agg2#78, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#72 AS agg3#79, avg(cast(cs_sales_price#7 as decimal(12,2)))#73 AS agg4#80, avg(cast(cs_net_profit#9 as decimal(12,2)))#74 AS agg5#81, avg(cast(c_birth_year#26 as decimal(12,2)))#75 AS agg6#82, avg(cast(cd_dep_count#13 as decimal(12,2)))#76 AS agg7#83] +Input [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] +Keys [5]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#27 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#71, avg(cast(cs_list_price#5 as decimal(12,2)))#72, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#73, avg(cast(cs_sales_price#6 as decimal(12,2)))#74, avg(cast(cs_net_profit#8 as decimal(12,2)))#75, avg(cast(c_birth_year#27 as decimal(12,2)))#76, avg(cast(cd_dep_count#14 as decimal(12,2)))#77] +Results [11]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, avg(cast(cs_quantity#4 as decimal(12,2)))#71 AS agg1#78, avg(cast(cs_list_price#5 as decimal(12,2)))#72 AS agg2#79, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#73 AS agg3#80, avg(cast(cs_sales_price#6 as decimal(12,2)))#74 AS agg4#81, avg(cast(cs_net_profit#8 as decimal(12,2)))#75 AS agg5#82, avg(cast(c_birth_year#27 as decimal(12,2)))#76 AS agg6#83, avg(cast(cd_dep_count#14 as decimal(12,2)))#77 AS agg7#84] (53) TakeOrderedAndProject -Input [11]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, agg1#77, agg2#78, agg3#79, agg4#80, agg5#81, agg6#82, agg7#83] -Arguments: 100, [ca_country#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_county#39 ASC NULLS FIRST, i_item_id#36 ASC NULLS FIRST], [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, agg1#77, agg2#78, agg3#79, agg4#80, agg5#81, agg6#82, agg7#83] +Input [11]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] +Arguments: 100, [ca_country#38 ASC NULLS FIRST, ca_state#39 ASC NULLS FIRST, ca_county#40 ASC NULLS FIRST, i_item_id#37 ASC NULLS FIRST], [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (54) + + +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#16] + 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 8069d43c3451a..8c76e7cab3310 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt @@ -54,34 +54,34 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - 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 4627bc19f25f0..89dfa65b4aa37 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt @@ -4,248 +4,248 @@ TakeOrderedAndProject (45) +- Exchange (43) +- * HashAggregate (42) +- * Project (41) - +- * 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_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#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 [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 4] +Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] + +(3) Filter [codegen id : 4] +Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] +Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) + +(4) Scan parquet default.item +Output [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] + +(6) Filter [codegen id : 1] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 8)) AND isnotnull(i_item_sk#6)) -(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#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -(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#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(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#2] +Right keys [1]: [i_item_sk#6] +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_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [10]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -(6) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] + +(13) Filter [codegen id : 2] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_moy#15) AND isnotnull(d_year#14)) AND (d_moy#15 = 11)) AND (d_year#14 = 1998)) AND isnotnull(d_date_sk#13)) -(7) ColumnarToRow -Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +(14) Project [codegen id : 2] +Output [1]: [d_date_sk#13] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -(8) Filter -Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -Condition : (((isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) AND isnotnull(ss_customer_sk#7)) AND isnotnull(ss_store_sk#8)) +(15) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] Join condition: None -(10) Project [codegen id : 3] -Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -Input [6]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +(17) Project [codegen id : 4] +Output [7]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [9]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, d_date_sk#13] -(11) Scan parquet default.store -Output [2]: [s_store_sk#10, s_zip#11] +(18) Scan parquet default.store +Output [2]: [s_store_sk#17, s_zip#18] 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#10, s_zip#11] +(19) ColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#17, s_zip#18] -(13) Filter [codegen id : 2] -Input [2]: [s_store_sk#10, s_zip#11] -Condition : (isnotnull(s_zip#11) AND isnotnull(s_store_sk#10)) +(20) Filter [codegen id : 3] +Input [2]: [s_store_sk#17, s_zip#18] +Condition : (isnotnull(s_zip#18) AND isnotnull(s_store_sk#17)) -(14) BroadcastExchange -Input [2]: [s_store_sk#10, s_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(21) BroadcastExchange +Input [2]: [s_store_sk#17, s_zip#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#8] -Right keys [1]: [s_store_sk#10] +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#17] Join condition: None -(16) Project [codegen id : 3] -Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] -Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9, s_store_sk#10, s_zip#11] +(23) Project [codegen id : 4] +Output [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Input [9]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_store_sk#17, s_zip#18] -(17) Exchange -Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] -Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#13] +(24) Exchange +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: hashpartitioning(ss_customer_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] -(18) Sort [codegen id : 4] -Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] -Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 +(25) Sort [codegen id : 5] +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: [ss_customer_sk#3 ASC NULLS FIRST], false, 0 -(19) Scan parquet default.customer -Output [2]: [c_customer_sk#14, c_current_addr_sk#15] +(26) Scan parquet default.customer +Output [2]: [c_customer_sk#21, c_current_addr_sk#22] 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#14, c_current_addr_sk#15] +(27) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -(21) Filter [codegen id : 5] -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) +(28) Filter [codegen id : 6] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) -(22) Exchange -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] +(29) Exchange +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: hashpartitioning(c_current_addr_sk#22, 5), ENSURE_REQUIREMENTS, [id=#23] -(23) Sort [codegen id : 6] -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 7] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_zip#18] +(31) Scan parquet default.customer_address +Output [2]: [ca_address_sk#24, ca_zip#25] 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#17, ca_zip#18] +(32) ColumnarToRow [codegen id : 8] +Input [2]: [ca_address_sk#24, ca_zip#25] -(26) Filter [codegen id : 7] -Input [2]: [ca_address_sk#17, ca_zip#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) +(33) Filter [codegen id : 8] +Input [2]: [ca_address_sk#24, ca_zip#25] +Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) -(27) Exchange -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: hashpartitioning(ca_address_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +(34) Exchange +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 +(35) Sort [codegen id : 9] +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#15] -Right keys [1]: [ca_address_sk#17] +(36) SortMergeJoin [codegen id : 10] +Left keys [1]: [c_current_addr_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None -(30) Project [codegen id : 9] -Output [2]: [c_customer_sk#14, ca_zip#18] -Input [4]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_zip#18] - -(31) Exchange -Input [2]: [c_customer_sk#14, ca_zip#18] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] - -(32) Sort [codegen id : 10] -Input [2]: [c_customer_sk#14, ca_zip#18] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(37) Project [codegen id : 10] +Output [2]: [c_customer_sk#21, ca_zip#25] +Input [4]: [c_customer_sk#21, c_current_addr_sk#22, ca_address_sk#24, ca_zip#25] -(33) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#7] -Right keys [1]: [c_customer_sk#14] -Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#11, 1, 5)) +(38) Exchange +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: hashpartitioning(c_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#27] -(34) Project [codegen id : 12] -Output [2]: [ss_item_sk#6, ss_ext_sales_price#9] -Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11, c_customer_sk#14, ca_zip#18] +(39) Sort [codegen id : 11] +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.item -Output [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(36) ColumnarToRow [codegen id : 11] -Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] - -(37) Filter [codegen id : 11] -Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] -Condition : ((isnotnull(i_manager_id#26) AND (i_manager_id#26 = 8)) AND isnotnull(i_item_sk#21)) - -(38) Project [codegen id : 11] -Output [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] - -(39) BroadcastExchange -Input [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] - -(40) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#21] -Join condition: None +(40) SortMergeJoin [codegen id : 12] +Left keys [1]: [ss_customer_sk#3] +Right keys [1]: [c_customer_sk#21] +Join condition: NOT (substr(ca_zip#25, 1, 5) = substr(s_zip#18, 1, 5)) (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Input [7]: [ss_item_sk#6, ss_ext_sales_price#9, i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Output [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Input [9]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18, c_customer_sk#21, ca_zip#25] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] -Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#9))] +Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] (43) Exchange -Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] -Arguments: hashpartitioning(i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Arguments: hashpartitioning(i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, 5), ENSURE_REQUIREMENTS, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] -Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#9))#31] -Results [5]: [i_brand_id#22 AS brand_id#32, i_brand#23 AS brand#33, i_manufact_id#24, i_manufact#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#9))#31,17,2) AS ext_price#34] +Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#31] +Results [5]: [i_brand_id#7 AS brand_id#32, i_brand#8 AS brand#33, i_manufact_id#9, i_manufact#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS ext_price#34] (45) TakeOrderedAndProject -Input [5]: [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] -Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#24 ASC NULLS FIRST, i_manufact#25 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] +Input [5]: [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] +Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#9 ASC NULLS FIRST, i_manufact#10 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] 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 b6441c5fe72c1..05fa3f82e27df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -6,71 +6,71 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (12) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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] + SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] + InputAdapter + WholeStageCodegen (5) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #2 + WholeStageCodegen (4) + Project [ss_customer_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + 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.date_dim [d_date_sk,d_year,d_moy] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] 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] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (8) - Sort [ca_address_sk] + 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 2ef3660bc7ba4..a5b9623cbd96e 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 @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -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_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +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#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] 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#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] 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 1e8ab18f5e21a..965b68218eb0a 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,34 +14,34 @@ 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_sold_date_sk,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 (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - 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] #4 + 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 #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-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index 093c4eed6cf11..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 @@ -1,567 +1,587 @@ == Physical Plan == -* Project (48) -+- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (18) - : : +- * SortMergeJoin Inner (17) - : : :- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- * Sort (16) - : : +- Exchange (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet default.customer (12) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildLeft (27) - : :- BroadcastExchange (23) - : : +- * Project (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.store (19) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_returns (34) +* Project (50) ++- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildLeft (28) + : :- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.store (20) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet default.customer_address (25) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.store_returns (35) (1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (3) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(4) Project [codegen id : 2] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] + +(5) Scan parquet default.item +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : ((isnotnull(i_color#9) AND (i_color#9 = pale)) AND isnotnull(i_item_sk#6)) +(7) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) -(7) BroadcastExchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(8) BroadcastExchange +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] -(8) BroadcastHashJoin [codegen id : 2] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#7] Join condition: None -(9) Project [codegen id : 2] -Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(10) Project [codegen id : 2] +Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(10) Exchange -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] +(11) Exchange +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#14] -(11) Sort [codegen id : 3] -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(12) Sort [codegen id : 3] +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(13) Scan parquet default.customer +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(14) ColumnarToRow [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(14) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(15) Filter [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) -(15) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(16) Exchange +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(16) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(17) Sort [codegen id : 5] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Right keys [1]: [c_customer_sk#15] Join condition: None -(18) Project [codegen id : 8] -Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(19) Project [codegen id : 8] +Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(19) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(20) Scan parquet default.store +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(21) ColumnarToRow [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(21) Filter [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(22) Filter [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) -(22) Project [codegen id : 6] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(23) Project [codegen id : 6] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(23) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#24] +(24) BroadcastExchange +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#25] -(24) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(25) Scan parquet default.customer_address +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(26) ColumnarToRow +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -(26) Filter -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(27) Filter +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) -(27) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_zip#23] -Right keys [1]: [ca_zip#26] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_zip#24] +Right keys [1]: [ca_zip#27] Join condition: None -(28) Project [codegen id : 7] -Output [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, ca_zip#26, ca_country#27] +(29) Project [codegen id : 7] +Output [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Input [7]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24, ca_state#26, ca_zip#27, ca_country#28] -(29) BroadcastExchange -Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] +(30) BroadcastExchange +Input [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#29] -(30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ss_store_sk#3, c_birth_country#17] -Right keys [2]: [s_store_sk#19, upper(ca_country#27)] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [ss_store_sk#3, c_birth_country#18] +Right keys [2]: [s_store_sk#20, upper(ca_country#28)] Join condition: None -(31) Project [codegen id : 8] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] +(32) Project [codegen id : 8] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] -(32) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] +(33) Exchange +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#30] -(33) Sort [codegen id : 9] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] +(34) Sort [codegen id : 9] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(35) Scan parquet default.store_returns +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] + +(37) Filter [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) -(36) Filter [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(38) Project [codegen id : 10] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -(37) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(38) Sort [codegen id : 11] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None -(40) Project [codegen id : 12] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +(42) Project [codegen id : 12] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26, sr_item_sk#31, sr_ticket_number#32] -(41) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(43) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#33] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -(42) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] +(44) Exchange +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] -(43) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(45) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#36] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#20, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#36,17,2) AS netpaid#37] - -(44) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#20, netpaid#37] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [partial_sum(netpaid#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] - -(45) Exchange -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] - -(46) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [sum(netpaid#37)] -Aggregate Attributes [1]: [sum(netpaid#37)#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum(netpaid#37)#43 AS paid#44, sum(netpaid#37)#43 AS sum(netpaid#37)#45] - -(47) Filter [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] -Condition : (isnotnull(sum(netpaid#37)#45) AND (cast(sum(netpaid#37)#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) - -(48) Project [codegen id : 14] -Output [4]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#21, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] + +(46) HashAggregate [codegen id : 13] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#21, netpaid#39] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] + +(47) Exchange +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, 5), ENSURE_REQUIREMENTS, [id=#44] + +(48) HashAggregate [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] + +(49) Filter [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] +Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) + +(50) Project [codegen id : 14] +Output [4]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * SortMergeJoin Inner (93) - :- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- * SortMergeJoin Inner (75) - : : :- * Sort (69) - : : : +- Exchange (68) - : : : +- * Project (67) - : : : +- * SortMergeJoin Inner (66) - : : : :- * Sort (60) - : : : : +- Exchange (59) - : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : :- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store_sales (49) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet default.store (52) - : : : +- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Sort (74) - : : +- Exchange (73) - : : +- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet default.item (70) - : +- * Sort (83) - : +- Exchange (82) - : +- * Filter (81) - : +- * ColumnarToRow (80) - : +- Scan parquet default.customer_address (79) - +- * Sort (92) - +- Exchange (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.store_returns (88) - - -(49) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * SortMergeJoin Inner (97) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (81) + : : +- Exchange (80) + : : +- * Project (79) + : : +- * SortMergeJoin Inner (78) + : : :- * Sort (72) + : : : +- Exchange (71) + : : : +- * Project (70) + : : : +- * SortMergeJoin Inner (69) + : : : :- * Sort (63) + : : : : +- Exchange (62) + : : : : +- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (54) + : : : : : +- * Filter (53) + : : : : : +- * ColumnarToRow (52) + : : : : : +- Scan parquet default.store_sales (51) + : : : : +- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store (55) + : : : +- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (77) + : : +- Exchange (76) + : : +- * Filter (75) + : : +- * ColumnarToRow (74) + : : +- Scan parquet default.customer (73) + : +- * Sort (86) + : +- Exchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer_address (82) + +- * Sort (96) + +- Exchange (95) + +- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.store_returns (91) + + +(51) Scan parquet default.store_sales +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +(52) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -(51) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) +(53) Filter [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) -(52) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(54) Project [codegen id : 2] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] + +(55) Scan parquet default.store +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(56) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(54) Filter [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(57) Filter [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) -(55) Project [codegen id : 1] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(58) Project [codegen id : 1] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(56) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +(59) BroadcastExchange +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#19] +(60) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None -(58) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] +(61) Project [codegen id : 2] +Output [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] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -(59) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] +(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_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] -(60) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(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_item_sk#50 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(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 -(62) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(63) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(66) Filter [codegen id : 4] +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) -(64) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] +(67) Exchange +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] -(65) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 5] +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 -(66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +(69) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None -(67) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Project [codegen id : 6] +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] -(68) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] +(71) Exchange +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] -(69) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(72) Sort [codegen id : 7] +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 -(70) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(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 -(71) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(74) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(72) Filter [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +(75) Filter [codegen id : 8] +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)) -(73) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] +(76) Exchange +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] -(74) Sort [codegen id : 9] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +(77) Sort [codegen id : 9] +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 -(75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +(78) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None -(76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(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, 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] -(77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] +(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, 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] -(78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 +(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, 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 -(79) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(82) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(83) ColumnarToRow [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(81) Filter [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(84) Filter [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(82) Exchange -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] +(85) Exchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] -(83) Sort [codegen id : 13] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +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 -(84) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#17, s_zip#23] -Right keys [2]: [upper(ca_country#27), ca_zip#26] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None -(85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] +(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, 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] -(86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +(89) Exchange +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] -(87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +(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, 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 -(88) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(91) Scan parquet default.store_returns +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(90) Filter [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(93) Filter [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) -(91) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] +(94) Project [codegen id : 16] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(92) Sort [codegen id : 17] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(95) Exchange +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] -(93) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] -Join condition: None - -(94) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] - -(95) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#57] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] +(96) Sort [codegen id : 17] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(96) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] - -(97) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#60] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#60,17,2) AS netpaid#37] +(97) SortMergeJoin [codegen id : 18] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Join condition: None -(98) HashAggregate [codegen id : 19] -Input [1]: [netpaid#37] +(98) Project [codegen id : 18] +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#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#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#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#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] + +(102) HashAggregate [codegen id : 19] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#37)] -Aggregate Attributes [2]: [sum#61, count#62] -Results [2]: [sum#63, count#64] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(99) Exchange -Input [2]: [sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +(103) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(100) HashAggregate [codegen id : 20] -Input [2]: [sum#63, count#64] +(104) HashAggregate [codegen id : 20] +Input [2]: [sum#92, count#93] Keys: [] -Functions [1]: [avg(netpaid#37)] -Aggregate Attributes [1]: [avg(netpaid#37)#66] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#37)#66)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#67] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 7de562c5d59a1..10f874f8f5543 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] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [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 273950bed3546..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 @@ -1,567 +1,587 @@ == Physical Plan == -* Project (48) -+- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (18) - : : +- * SortMergeJoin Inner (17) - : : :- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- * Sort (16) - : : +- Exchange (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet default.customer (12) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildLeft (27) - : :- BroadcastExchange (23) - : : +- * Project (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.store (19) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_returns (34) +* Project (50) ++- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildLeft (28) + : :- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.store (20) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet default.customer_address (25) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.store_returns (35) (1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (3) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(4) Project [codegen id : 2] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] + +(5) Scan parquet default.item +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : ((isnotnull(i_color#9) AND (i_color#9 = chiffon)) AND isnotnull(i_item_sk#6)) +(7) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) -(7) BroadcastExchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(8) BroadcastExchange +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] -(8) BroadcastHashJoin [codegen id : 2] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#7] Join condition: None -(9) Project [codegen id : 2] -Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(10) Project [codegen id : 2] +Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(10) Exchange -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] +(11) Exchange +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#14] -(11) Sort [codegen id : 3] -Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(12) Sort [codegen id : 3] +Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(13) Scan parquet default.customer +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(14) ColumnarToRow [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(14) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(15) Filter [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) -(15) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(16) Exchange +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(16) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(17) Sort [codegen id : 5] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Right keys [1]: [c_customer_sk#15] Join condition: None -(18) Project [codegen id : 8] -Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(19) Project [codegen id : 8] +Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(19) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(20) Scan parquet default.store +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(21) ColumnarToRow [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(21) Filter [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(22) Filter [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) -(22) Project [codegen id : 6] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(23) Project [codegen id : 6] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(23) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#24] +(24) BroadcastExchange +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#25] -(24) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(25) Scan parquet default.customer_address +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(26) ColumnarToRow +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -(26) Filter -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(27) Filter +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) -(27) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_zip#23] -Right keys [1]: [ca_zip#26] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_zip#24] +Right keys [1]: [ca_zip#27] Join condition: None -(28) Project [codegen id : 7] -Output [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, ca_zip#26, ca_country#27] +(29) Project [codegen id : 7] +Output [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Input [7]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24, ca_state#26, ca_zip#27, ca_country#28] -(29) BroadcastExchange -Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] +(30) BroadcastExchange +Input [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#29] -(30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ss_store_sk#3, c_birth_country#17] -Right keys [2]: [s_store_sk#19, upper(ca_country#27)] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [ss_store_sk#3, c_birth_country#18] +Right keys [2]: [s_store_sk#20, upper(ca_country#28)] Join condition: None -(31) Project [codegen id : 8] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] +(32) Project [codegen id : 8] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] -(32) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] +(33) Exchange +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#30] -(33) Sort [codegen id : 9] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] +(34) Sort [codegen id : 9] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(35) Scan parquet default.store_returns +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] + +(37) Filter [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) -(36) Filter [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(38) Project [codegen id : 10] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -(37) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(38) Sort [codegen id : 11] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None -(40) Project [codegen id : 12] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +(42) Project [codegen id : 12] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26, sr_item_sk#31, sr_ticket_number#32] -(41) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(43) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#33] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -(42) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] +(44) Exchange +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] -(43) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] +(45) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#36] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#20, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#36,17,2) AS netpaid#37] - -(44) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#20, netpaid#37] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [partial_sum(netpaid#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] - -(45) Exchange -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] - -(46) HashAggregate [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [sum(netpaid#37)] -Aggregate Attributes [1]: [sum(netpaid#37)#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum(netpaid#37)#43 AS paid#44, sum(netpaid#37)#43 AS sum(netpaid#37)#45] - -(47) Filter [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] -Condition : (isnotnull(sum(netpaid#37)#45) AND (cast(sum(netpaid#37)#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) - -(48) Project [codegen id : 14] -Output [4]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#21, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] + +(46) HashAggregate [codegen id : 13] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#21, netpaid#39] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] + +(47) Exchange +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, 5), ENSURE_REQUIREMENTS, [id=#44] + +(48) HashAggregate [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] + +(49) Filter [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] +Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) + +(50) Project [codegen id : 14] +Output [4]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * SortMergeJoin Inner (93) - :- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- * SortMergeJoin Inner (75) - : : :- * Sort (69) - : : : +- Exchange (68) - : : : +- * Project (67) - : : : +- * SortMergeJoin Inner (66) - : : : :- * Sort (60) - : : : : +- Exchange (59) - : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : :- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store_sales (49) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet default.store (52) - : : : +- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Sort (74) - : : +- Exchange (73) - : : +- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet default.item (70) - : +- * Sort (83) - : +- Exchange (82) - : +- * Filter (81) - : +- * ColumnarToRow (80) - : +- Scan parquet default.customer_address (79) - +- * Sort (92) - +- Exchange (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.store_returns (88) - - -(49) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * SortMergeJoin Inner (97) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (81) + : : +- Exchange (80) + : : +- * Project (79) + : : +- * SortMergeJoin Inner (78) + : : :- * Sort (72) + : : : +- Exchange (71) + : : : +- * Project (70) + : : : +- * SortMergeJoin Inner (69) + : : : :- * Sort (63) + : : : : +- Exchange (62) + : : : : +- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (54) + : : : : : +- * Filter (53) + : : : : : +- * ColumnarToRow (52) + : : : : : +- Scan parquet default.store_sales (51) + : : : : +- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store (55) + : : : +- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (77) + : : +- Exchange (76) + : : +- * Filter (75) + : : +- * ColumnarToRow (74) + : : +- Scan parquet default.customer (73) + : +- * Sort (86) + : +- Exchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer_address (82) + +- * Sort (96) + +- Exchange (95) + +- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.store_returns (91) + + +(51) Scan parquet default.store_sales +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +(52) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -(51) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) +(53) Filter [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) -(52) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(54) Project [codegen id : 2] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] + +(55) Scan parquet default.store +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(56) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(54) Filter [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(57) Filter [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) -(55) Project [codegen id : 1] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(58) Project [codegen id : 1] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(56) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +(59) BroadcastExchange +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#19] +(60) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None -(58) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] +(61) Project [codegen id : 2] +Output [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] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -(59) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] +(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_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] -(60) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(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_item_sk#50 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(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 -(62) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(63) Filter [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +(66) Filter [codegen id : 4] +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) -(64) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] +(67) Exchange +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] -(65) Sort [codegen id : 5] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 5] +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 -(66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +(69) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None -(67) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Project [codegen id : 6] +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] -(68) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] +(71) Exchange +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] -(69) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(72) Sort [codegen id : 7] +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 -(70) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(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 -(71) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(74) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(72) Filter [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +(75) Filter [codegen id : 8] +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)) -(73) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] +(76) Exchange +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] -(74) Sort [codegen id : 9] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +(77) Sort [codegen id : 9] +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 -(75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +(78) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] Join condition: None -(76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(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, 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] -(77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] +(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, 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] -(78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 +(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, 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 -(79) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(82) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(83) ColumnarToRow [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(81) Filter [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(84) Filter [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(82) Exchange -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] +(85) Exchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] -(83) Sort [codegen id : 13] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +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 -(84) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#17, s_zip#23] -Right keys [2]: [upper(ca_country#27), ca_zip#26] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None -(85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] +(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, 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] -(86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +(89) Exchange +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] -(87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +(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, 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 -(88) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(91) Scan parquet default.store_returns +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(90) Filter [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(93) Filter [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) -(91) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] +(94) Project [codegen id : 16] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(92) Sort [codegen id : 17] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(95) Exchange +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] -(93) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] -Join condition: None - -(94) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] - -(95) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#57] -Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] +(96) Sort [codegen id : 17] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(96) Exchange -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] - -(97) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Keys [10]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#60] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#60,17,2) AS netpaid#37] +(97) SortMergeJoin [codegen id : 18] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Join condition: None -(98) HashAggregate [codegen id : 19] -Input [1]: [netpaid#37] +(98) Project [codegen id : 18] +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#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#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#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#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] + +(102) HashAggregate [codegen id : 19] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#37)] -Aggregate Attributes [2]: [sum#61, count#62] -Results [2]: [sum#63, count#64] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(99) Exchange -Input [2]: [sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +(103) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(100) HashAggregate [codegen id : 20] -Input [2]: [sum#63, count#64] +(104) HashAggregate [codegen id : 20] +Input [2]: [sum#92, count#93] Keys: [] -Functions [1]: [avg(netpaid#37)] -Aggregate Attributes [1]: [avg(netpaid#37)#66] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#37)#66)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#67] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 7de562c5d59a1..10f874f8f5543 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] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [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 3100e574e60e3..a531797a20c7e 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 @@ -224,76 +224,76 @@ Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 + +(40) SortMergeJoin [codegen id : 12] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] +Join condition: None + +(41) Project [codegen id : 12] +Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] + +(42) Exchange +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#31] + +(43) Sort [codegen id : 13] +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 -(40) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +(44) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] - -(42) Filter [codegen id : 13] -Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Condition : ((isnotnull(cs_bill_customer_sk#32) AND isnotnull(cs_item_sk#33)) AND isnotnull(cs_sold_date_sk#31)) +(45) ColumnarToRow [codegen id : 15] +Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -(43) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#35] +(46) Filter [codegen id : 15] +Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Condition : ((isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) AND isnotnull(cs_sold_date_sk#32)) -(44) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cs_sold_date_sk#31] -Right keys [1]: [d_date_sk#35] -Join condition: None - -(45) Project [codegen id : 13] -Output [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Input [5]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34, d_date_sk#35] - -(46) Exchange -Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#36] +(47) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#36] -(47) Sort [codegen id : 14] -Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] -Arguments: [cast(cs_bill_customer_sk#32 as bigint) ASC NULLS FIRST, cast(cs_item_sk#33 as bigint) ASC NULLS FIRST], false, 0 - -(48) SortMergeJoin [codegen id : 15] -Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] -Right keys [2]: [cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint)] +(48) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#32] +Right keys [1]: [d_date_sk#36] 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#34] -Input [7]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Output [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Input [5]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, d_date_sk#36] (50) Exchange -Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#37] +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=#37] (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#34] -Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 +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 (52) SortMergeJoin [codegen id : 17] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] +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)] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [13]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#34))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] Aggregate Attributes [3]: [sum#38, sum#39, sum#40] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#34))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#34))#47] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#35))#47] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] (57) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_profit#48, store_returns_loss#49, catalog_sales_profit#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index 9b53cdaa5dc67..ad9fa718ff2bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [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_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.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_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter - 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] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + WholeStageCodegen (16) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (15) + Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter - WholeStageCodegen (14) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (13) - Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] - InputAdapter - ReusedExchange [d_date_sk] #9 + 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 cb8522545f1d3..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) @@ -69,310 +69,327 @@ TakeOrderedAndProject (67) (1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] +Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] (3) Filter [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_item_sk#2)) +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] +Output [3]: [d_date_sk#6, d_year#7, d_moy#8] 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,5), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 5)) AND isnotnull(d_date_sk#5)) +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 = 5)) AND isnotnull(d_date_sk#6)) (7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Output [1]: [d_date_sk#6] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] +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#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(13) Filter [codegen id : 2] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(14) Project [codegen id : 2] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] - -(15) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None - -(17) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] -Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] - -(18) Scan parquet default.item -Output [2]: [i_item_sk#12, i_manufact_id#13] +(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#12, i_manufact_id#13] +(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#12, i_manufact_id#13] -Condition : isnotnull(i_item_sk#12) +(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#14, i_manufact_id#13] +(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)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] +(15) ColumnarToRow [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] -(23) Filter [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] -Condition : (isnotnull(i_category#14) AND (i_category#14 = 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#13 AS i_manufact_id#13#15] -Input [2]: [i_category#14, i_manufact_id#13] +(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#13#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(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#13] -Right keys [1]: [i_manufact_id#13#15] +(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#12, i_manufact_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +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#2] -Right keys [1]: [i_item_sk#12] +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#4, i_manufact_id#13] -Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_manufact_id#13] +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#4, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [i_manufact_id#13, sum#19] +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#11, sum#20] (31) Exchange -Input [2]: [i_manufact_id#13, sum#19] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#20] +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#13, sum#19] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] +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#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_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] +Output [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(cs_sold_date_sk#27 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, 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, cs_sold_date_sk#27] (35) Filter [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Condition : ((isnotnull(cs_sold_date_sk#23) AND isnotnull(cs_bill_addr_sk#24)) AND isnotnull(cs_item_sk#25)) +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] +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#9] +(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#9] +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#9] +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#12, i_manufact_id#13] +Output [1]: [ca_address_sk#31] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#12] +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#13] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_manufact_id#13] +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#13] -Keys [1]: [i_manufact_id#13] +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#27] -Results [2]: [i_manufact_id#13, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#30, sum#33] (46) Exchange -Input [2]: [i_manufact_id#13, sum#28] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +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#13, sum#28] -Keys [1]: [i_manufact_id#13] +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))#30] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +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_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Condition : ((isnotnull(ws_sold_date_sk#32) AND isnotnull(ws_bill_addr_sk#34)) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] +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#9] +(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#34] -Right keys [1]: [ca_address_sk#9] +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#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] +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#12, i_manufact_id#13] +Output [1]: [ca_address_sk#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#12] +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#35, i_manufact_id#13] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_manufact_id#13] +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#35, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_manufact_id#13, sum#37] +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#43, sum#46] (61) Exchange -Input [2]: [i_manufact_id#13, sum#37] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#38] +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#13, sum#37] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] +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#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#13, total_sales#22] -Keys [1]: [i_manufact_id#13] -Functions [1]: [partial_sum(total_sales#22)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#13, sum#43, isEmpty#44] +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#11, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#45] +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#13, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#13] -Functions [1]: [sum(total_sales#22)] -Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_manufact_id#13, sum(total_sales#22)#46 AS total_sales#47] +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#11, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#13, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] +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 ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +ReusedExchange (68) + + +(68) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] + +Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 + 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 14787f0bbce7b..410def2466e1a 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_sold_date_sk,ss_addr_sk,ss_item_sk] @@ -33,28 +33,28 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (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 @@ -62,9 +62,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (11) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_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_sold_date_sk,cs_bill_addr_sk,cs_item_sk] @@ -74,9 +74,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [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 @@ -84,9 +84,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_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_sold_date_sk,ws_bill_addr_sk,ws_item_sk] @@ -96,6 +96,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [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 6492918d3aa13..036de304b72cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt @@ -6,117 +6,117 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner 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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#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 [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), 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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) -(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#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct -(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#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(6) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 1)) AND isnotnull(i_item_sk#4)) -(7) ColumnarToRow -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +(7) Project [codegen id : 1] +Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(8) Filter -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) +(8) BroadcastExchange +Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None (10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7] -Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] 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#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 2000)) AND isnotnull(d_date_sk#9)) (14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Output [2]: [d_date_sk#9, d_year#10] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#8] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Input [6]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] +Output [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [6]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9, d_year#10] (18) HashAggregate [codegen id : 3] -Input [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] +Input [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Results [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] (19) Exchange -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Arguments: hashpartitioning(d_year#10, i_brand#6, i_brand_id#5, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] -Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] -Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] +Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [4]: [d_year#10, i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] +Input [4]: [d_year#10, brand_id#17, brand#18, ext_price#19] +Arguments: 100, [d_year#10 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#10, brand_id#17, brand#18, ext_price#19] 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 f4aaf3df75135..8ed500d84390c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 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 b8d8aa358d532..cca43a4232a8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt @@ -6,115 +6,115 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner 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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#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 [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), 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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) -(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#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] +ReadSchema: struct -(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#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(6) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 28)) AND isnotnull(i_item_sk#4)) -(7) ColumnarToRow -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +(7) Project [codegen id : 1] +Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(8) Filter -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) +(8) BroadcastExchange +Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None (10) Project [codegen id : 3] -Output [2]: [ss_item_sk#6, ss_ext_sales_price#7] -Input [4]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] 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#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (13) Filter [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 28)) AND isnotnull(i_item_sk#8)) +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 1999)) AND isnotnull(d_date_sk#9)) (14) Project [codegen id : 2] -Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Output [1]: [d_date_sk#9] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#8] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Input [5]: [ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] +Output [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Input [5]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9] (18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] -Keys [2]: [i_brand#10, i_brand_id#9] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] +Input [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] +Keys [2]: [i_brand#6, i_brand_id#5] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#10, i_brand_id#9, sum#14] +Results [3]: [i_brand#6, i_brand_id#5, sum#14] (19) Exchange -Input [3]: [i_brand#10, i_brand_id#9, sum#14] -Arguments: hashpartitioning(i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [i_brand#6, i_brand_id#5, sum#14] +Arguments: hashpartitioning(i_brand#6, i_brand_id#5, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#10, i_brand_id#9, sum#14] -Keys [2]: [i_brand#10, i_brand_id#9] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] -Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] +Input [3]: [i_brand#6, i_brand_id#5, sum#14] +Keys [2]: [i_brand#6, i_brand_id#5] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [3]: [i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt index 4f375c80678e8..b0d0e0d809441 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 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 3007b11a1a860..3a61d77f674f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (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) @@ -98,7 +98,7 @@ Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnot Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -106,7 +106,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (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)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] @@ -213,95 +213,95 @@ Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 10] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Exchange -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] - -(34) Sort [codegen id : 9] -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 - -(35) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(37) Filter [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(38) Project [codegen id : 10] -Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(39) BroadcastExchange -Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_week_seq#29] +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow -Input [2]: [d_date_sk#28, d_week_seq#29] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(42) Filter -Input [2]: [d_date_sk#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#29] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None -(44) Project [codegen id : 11] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] +(42) Project [codegen id : 9] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] + +(43) BroadcastExchange +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d_date#23 + 5 days) + +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -(45) BroadcastExchange -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] +(46) Exchange +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] -(46) Scan parquet default.inventory +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(47) ColumnarToRow +(49) ColumnarToRow [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(48) Filter +(50) Filter [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [d_date_sk#28] -Right keys [1]: [inv_date_sk#31] -Join condition: None - -(50) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] - (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] -Right keys [2]: [inv_item_sk#32, d_date_sk#23] -Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) +Left keys [2]: [cs_item_sk#5, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#31] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] 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 b88505ad7b9bc..918508787c4b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,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_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow + Exchange [cs_item_sk,d_date_sk] #3 + WholeStageCodegen (10) + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #6 + 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 #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] 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 #9 + WholeStageCodegen (9) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] 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_date_sk,inv_item_sk,inv_quantity_on_hand,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,inv_date_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,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 6813696266ac5..f89e4b8d093f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -1,343 +1,343 @@ == Physical Plan == TakeOrderedAndProject (61) +- * Project (60) - +- * SortMergeJoin Inner (59) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_returns (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer_address (13) - : +- BroadcastExchange (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * SortMergeJoin Inner (34) - : :- * Sort (31) - : : +- Exchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_returns (24) - : : +- ReusedExchange (27) - : +- * Sort (33) - : +- ReusedExchange (32) - +- * Sort (58) - +- Exchange (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Filter (50) - : +- * ColumnarToRow (49) - : +- Scan parquet default.customer (48) - +- BroadcastExchange (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.customer_address (51) - - -(1) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] + +- * 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 [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +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_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#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_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Condition : ((isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) AND isnotnull(cr_returning_customer_sk#2)) +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#5, d_year#6] +(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#5, d_year#6] +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#5, d_year#6] -Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_year#6] +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#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(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#1] -Right keys [1]: [d_date_sk#5] +(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#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] +(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#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#8] +(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#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: [cr_returning_addr_sk#3 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#9, ca_state#10] +(12) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#9, ca_state#10] +(13) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -(15) Filter [codegen id : 4] -Input [2]: [ca_address_sk#9, ca_state#10] -Condition : (isnotnull(ca_address_sk#9) AND isnotnull(ca_state#10)) +(14) Filter [codegen id : 5] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Condition : ((isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) AND isnotnull(cr_returning_customer_sk#22)) -(16) Exchange -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: hashpartitioning(ca_address_sk#9, 5), ENSURE_REQUIREMENTS, [id=#11] +(15) Scan parquet default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) Sort [codegen id : 5] -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 +(16) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#25, d_year#26] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None +(17) Filter [codegen id : 4] +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2000)) AND isnotnull(d_date_sk#25)) -(19) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] - -(20) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum#12] -Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] - -(21) Exchange -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] -Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] - -(22) HashAggregate [codegen id : 15] -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#15] -Results [3]: [cr_returning_customer_sk#2 AS ctr_customer_sk#16, ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#15,17,2) AS ctr_total_return#18] - -(23) Filter [codegen id : 15] -Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] -Condition : isnotnull(ctr_total_return#18) - -(24) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct +(18) Project [codegen id : 4] +Output [1]: [d_date_sk#25] +Input [2]: [d_date_sk#25, d_year#26] -(25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +(19) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(26) Filter [codegen id : 8] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Condition : (isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#25] +Join condition: None -(27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(21) Project [codegen id : 5] +Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None +(22) Exchange +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), ENSURE_REQUIREMENTS, [id=#28] -(29) Project [codegen id : 8] -Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] +(23) Sort [codegen id : 6] +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 -(30) Exchange -Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#19] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#7, ca_state#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct -(31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#7, ca_state#14] -(32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#9, ca_state#10] +(26) Filter [codegen id : 7] +Input [2]: [ca_address_sk#7, ca_state#14] +Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#14)) -(33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 +(27) Exchange +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: hashpartitioning(ca_address_sk#7, 5), ENSURE_REQUIREMENTS, [id=#29] -(34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 -(35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] - -(36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] - -(37) Exchange -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] -Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#22] - -(38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] -Keys [2]: [cr_returning_customer_sk#2, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#23] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#23,17,2) AS ctr_total_return#18] - -(39) HashAggregate [codegen id : 13] -Input [2]: [ctr_state#17, ctr_total_return#18] -Keys [1]: [ctr_state#17] -Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ctr_state#17, sum#26, count#27] - -(40) Exchange -Input [3]: [ctr_state#17, sum#26, count#27] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#28] - -(41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#17, sum#26, count#27] -Keys [1]: [ctr_state#17] -Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#29] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#29) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17 AS ctr_state#17#31] - -(42) Filter [codegen id : 14] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) - -(43) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#32] - -(44) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#31] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) - -(45) Project [codegen id : 15] -Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] - -(46) Exchange -Input [2]: [ctr_customer_sk#16, ctr_total_return#18] -Arguments: hashpartitioning(ctr_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#33] - -(47) Sort [codegen id : 16] -Input [2]: [ctr_customer_sk#16, ctr_total_return#18] -Arguments: [ctr_customer_sk#16 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.customer -Output [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [cr_returning_addr_sk#23] +Right keys [1]: [ca_address_sk#7] +Join condition: None -(49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +(30) Project [codegen id : 9] +Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] + +(31) HashAggregate [codegen id : 9] +Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] + +(32) Exchange +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] +Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), ENSURE_REQUIREMENTS, [id=#32] + +(33) HashAggregate [codegen id : 10] +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#33] +Results [3]: [cr_returning_customer_sk#22 AS ctr_customer_sk#34, ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#33,17,2) AS ctr_total_return#36] + +(34) Filter [codegen id : 10] +Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +Condition : isnotnull(ctr_total_return#36) + +(35) Exchange +Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +Arguments: hashpartitioning(ctr_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#37] + +(36) Sort [codegen id : 11] +Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +Arguments: [ctr_customer_sk#34 ASC NULLS FIRST], false, 0 + +(37) SortMergeJoin [codegen id : 20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ctr_customer_sk#34] +Join condition: None -(50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) +(38) Project [codegen id : 20] +Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36] +Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -(51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(39) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(40) ColumnarToRow [codegen id : 13] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -(53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +(41) Filter [codegen id : 13] +Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Condition : (isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) -(54) BroadcastExchange -Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +(42) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#25] -(55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#9] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#25] Join condition: None -(56) Project [codegen id : 18] -Output [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Input [18]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +(44) Project [codegen id : 13] +Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] -(57) Exchange -Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: hashpartitioning(c_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#51] +(45) Exchange +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] -(58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 14] +Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 -(59) SortMergeJoin [codegen id : 20] -Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#34] +(47) ReusedExchange [Reuses operator id: 27] +Output [2]: [ca_address_sk#7, ca_state#14] + +(48) Sort [codegen id : 16] +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 + +(49) SortMergeJoin [codegen id : 17] +Left keys [1]: [cr_returning_addr_sk#23] +Right keys [1]: [ca_address_sk#7] Join condition: None +(50) Project [codegen id : 17] +Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] + +(51) HashAggregate [codegen id : 17] +Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum#39] +Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] + +(52) Exchange +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] +Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), ENSURE_REQUIREMENTS, [id=#41] + +(53) HashAggregate [codegen id : 18] +Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] +Keys [2]: [cr_returning_customer_sk#22, ca_state#14] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#42] +Results [2]: [ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#42,17,2) AS ctr_total_return#36] + +(54) HashAggregate [codegen id : 18] +Input [2]: [ctr_state#35, ctr_total_return#36] +Keys [1]: [ctr_state#35] +Functions [1]: [partial_avg(ctr_total_return#36)] +Aggregate Attributes [2]: [sum#43, count#44] +Results [3]: [ctr_state#35, sum#45, count#46] + +(55) Exchange +Input [3]: [ctr_state#35, sum#45, count#46] +Arguments: hashpartitioning(ctr_state#35, 5), ENSURE_REQUIREMENTS, [id=#47] + +(56) HashAggregate [codegen id : 19] +Input [3]: [ctr_state#35, sum#45, count#46] +Keys [1]: [ctr_state#35] +Functions [1]: [avg(ctr_total_return#36)] +Aggregate Attributes [1]: [avg(ctr_total_return#36)#48] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#36)#48) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35 AS ctr_state#35#50] + +(57) Filter [codegen id : 19] +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) + +(58) BroadcastExchange +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#51] + +(59) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [ctr_state#35] +Right keys [1]: [ctr_state#35#50] +Join condition: (cast(ctr_total_return#36 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) + (60) Project [codegen id : 20] -Output [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] -Input [18]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] +Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] -Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, ca_street_number#40 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#42 ASC NULLS FIRST, ca_suite_number#43 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#46 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#49 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] +Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#36 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] 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 99677b6e39736..c603ab5194286 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,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] + SortMergeJoin [c_customer_sk,ctr_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #1 + WholeStageCodegen (2) + Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + InputAdapter + WholeStageCodegen (11) + Sort [ctr_customer_sk] + InputAdapter + Exchange [ctr_customer_sk] #3 + WholeStageCodegen (10) 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_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] @@ -31,74 +50,55 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter - BroadcastExchange #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 [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #7 - WholeStageCodegen (13) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #8 - WholeStageCodegen (12) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - SortMergeJoin [cr_returning_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (9) - Sort [cr_returning_addr_sk] - InputAdapter - Exchange [cr_returning_addr_sk] #9 - WholeStageCodegen (8) - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - WholeStageCodegen (11) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #5 InputAdapter - 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 [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] + InputAdapter + Exchange [ctr_state] #9 + WholeStageCodegen (18) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] 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_returned_date_sk,cr_returning_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + WholeStageCodegen (16) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #7 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 6bcbe470cec50..16589f89deb6e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -8,206 +8,206 @@ +- * BroadcastHashJoin Inner BuildRight (41) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (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_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#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 [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +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_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#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_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -Condition : ((isnotnull(cr_call_center_sk#3) AND isnotnull(cr_returned_date_sk#1)) AND isnotnull(cr_returning_customer_sk#2)) +(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#5, d_year#6, d_moy#7] +(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#5, d_year#6, d_moy#7] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] (6) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 11)) AND isnotnull(d_date_sk#5)) +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)) -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +(7) BroadcastExchange +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] +(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#2, cr_call_center_sk#3, cr_net_loss#4] -Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, d_date_sk#5] - -(11) BroadcastExchange -Input [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] +(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#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +(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#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +(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#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -Condition : (((isnotnull(c_customer_sk#10) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_current_cdemo_sk#11)) AND isnotnull(c_current_hdemo_sk#12)) +(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#2] -Right keys [1]: [c_customer_sk#10] +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#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -Input [7]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +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#14, hd_buy_potential#15] +(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#14, hd_buy_potential#15] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (19) Filter [codegen id : 3] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] -Condition : ((isnotnull(hd_buy_potential#15) AND StartsWith(hd_buy_potential#15, Unknown)) AND isnotnull(hd_demo_sk#14)) +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#14] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Output [1]: [ca_address_sk#12] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (21) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +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#12] -Right keys [1]: [hd_demo_sk#14] +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#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13] -Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13, hd_demo_sk#14] +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#17, ca_gmt_offset#18] +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +(25) ColumnarToRow [codegen id : 5] +Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -(26) Filter [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -7.00)) AND isnotnull(ca_address_sk#17)) +(26) Filter [codegen id : 5] +Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Condition : ((isnotnull(cr_call_center_sk#17) AND isnotnull(cr_returned_date_sk#15)) AND isnotnull(cr_returning_customer_sk#16)) -(27) Project [codegen id : 4] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +(27) Scan parquet default.date_dim +Output [3]: [d_date_sk#19, d_year#20, d_moy#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct -(28) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +(28) ColumnarToRow [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#13] -Right keys [1]: [ca_address_sk#17] -Join condition: None +(29) Filter [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 1998)) AND (d_moy#21 = 11)) AND isnotnull(d_date_sk#19)) -(30) Project [codegen id : 7] -Output [3]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11] -Input [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13, ca_address_sk#17] +(30) Project [codegen id : 4] +Output [1]: [d_date_sk#19] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(31) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] -ReadSchema: struct +(31) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(32) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +(32) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] +Join condition: None -(33) Filter [codegen id : 5] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((cd_marital_status#21 = M) AND (cd_education_status#22 = Unknown)) OR ((cd_marital_status#21 = W) AND (cd_education_status#22 = Advanced Degree))) AND isnotnull(cd_demo_sk#20)) +(33) Project [codegen id : 5] +Output [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Input [5]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18, d_date_sk#19] (34) BroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#11] -Right keys [1]: [cd_demo_sk#20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cr_returning_customer_sk#16] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] -Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18] +Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] @@ -228,31 +228,31 @@ Input [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#3] +Left keys [1]: [cr_call_center_sk#17] Right keys [1]: [cc_call_center_sk#24] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] -Input [8]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] +Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] +Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#4))] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#18))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] (44) Exchange -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] -Functions [1]: [sum(UnscaledValue(cr_net_loss#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#4))#32] -Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#4))#32,17,2) AS Returns_Loss#36] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] +Functions [1]: [sum(UnscaledValue(cr_net_loss#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#18))#32] +Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#18))#32,17,2) AS Returns_Loss#36] (46) Exchange Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index 6c8d629feed3e..87beb3b565cc1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -10,58 +10,58 @@ WholeStageCodegen (9) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] - Project [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_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + 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_returned_date_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] InputAdapter - 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 30dabdd2d5523..c005ef0f9ff1a 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 @@ -37,126 +37,126 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +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#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] 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#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) Exchange -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), true, [id=#25] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#25] (28) Sort [codegen id : 10] -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 (29) Project [codegen id : 10] -Output [6]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] +Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] 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 a180ed2b060b9..c6ce597f3c1bf 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,34 +18,34 @@ 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_sold_date_sk,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 (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - 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] #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] #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 #6 + 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 c1bf12b7c2c5a..60d89b18f8fc9 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 @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -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_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: hashpartitioning(ws_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +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#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] 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#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] 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 d8db515f84f68..cfb9973ef6983 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,34 +14,34 @@ 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_sold_date_sk,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 (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - 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] #4 + 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 #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/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt index f6c5258701525..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) @@ -162,716 +162,733 @@ TakeOrderedAndProject (160) (1) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (3) Filter [codegen id : 4] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = M)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#10)) +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = M)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (11) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#16, d_year#17] (13) Filter [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#16] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_year#17] (15) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (17) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (18) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#19, i_item_id#20] (20) Filter [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (21) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (23) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_sk#18, i_item_id#19] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#19, i_item_id#20] (24) Exchange -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 5] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (26) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true 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] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(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] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +(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] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +(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#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] - -(31) Sort [codegen id : 7] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#28] +(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#28] +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#28] -Condition : isnotnull(cd_demo_sk#28) +(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#28] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +(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#28] -Arguments: [cd_demo_sk#28 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#23] -Right keys [1]: [cd_demo_sk#28] +(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#22, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] +(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] + +(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] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#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 + +(39) ColumnarToRow [codegen id : 9] +Input [1]: [cd_demo_sk#34] -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(41) Filter [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(42) BroadcastExchange -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +(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#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +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#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] +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#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +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] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -Output [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +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#19, ca_country#33, ca_state#32, ca_county#31, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +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#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#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 5), ENSURE_REQUIREMENTS, [id=#71] +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#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#72, avg(agg2#37)#73, avg(agg3#38)#74, avg(agg4#39)#75, avg(agg5#40)#76, avg(agg6#41)#77, avg(agg7#42)#78] -Results [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] +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#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: 24] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] +(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] (53) Sort [codegen id : 19] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [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] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] Batched: true 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 (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) (61) BroadcastExchange -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_state#32, ca_country#33] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] (64) Exchange -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 +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] -Output [1]: [cd_demo_sk#88] +(66) ReusedExchange [Reuses operator id: 41] +Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#88] -Arguments: [cd_demo_sk#88 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#105] +Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#88] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#105] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33, cd_demo_sk#88] +Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] (70) Exchange -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#95] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#19, ca_country#33, ca_state#32, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] +Input [12]: [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, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#19, ca_country#33, ca_state#32, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103] -Results [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +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#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] (75) Exchange -Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#119, avg(agg2#37)#120, avg(agg3#38)#121, avg(agg4#39)#122, avg(agg5#40)#123, avg(agg6#41)#124, avg(agg7#42)#125] -Results [11]: [i_item_id#19, ca_country#33, ca_state#32, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +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)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] +Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] -(77) ReusedExchange [Reuses operator id: 24] -Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] +(77) ReusedExchange [Reuses operator id: unknown] +Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] (78) Sort [codegen id : 33] -Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +Arguments: [cs_bill_customer_sk#151 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] Batched: true 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 (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#30, ca_country#33] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [2]: [ca_address_sk#164, ca_country#166] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (87) BroadcastExchange -Input [2]: [ca_address_sk#30, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#134] +Input [2]: [ca_address_sk#164, ca_country#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#164] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_country#33] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] (90) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] -Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 +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] -Output [1]: [cd_demo_sk#136] +(92) ReusedExchange [Reuses operator id: 41] +Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#136] -Arguments: [cd_demo_sk#136 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#169] +Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#136] +Left keys [1]: [c_current_cdemo_sk#160] +Right keys [1]: [cd_demo_sk#169] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33, cd_demo_sk#136] +Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] (96) Exchange -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#151] +Right keys [1]: [c_customer_sk#159] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#19, ca_country#33, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#33] +Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] +Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#19, ca_country#33, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [2]: [i_item_id#19, ca_country#33] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149, sum#150, count#151] -Results [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#158, ca_country#166] +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#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] +Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] (101) Exchange -Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Arguments: hashpartitioning(i_item_id#19, ca_country#33, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Keys [2]: [i_item_id#19, ca_country#33] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#167, avg(agg2#37)#168, avg(agg3#38)#169, avg(agg4#39)#170, avg(agg5#40)#171, avg(agg6#41)#172, avg(agg7#42)#173] -Results [11]: [i_item_id#19, ca_country#33, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Keys [2]: [i_item_id#158, ca_country#166] +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)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] +Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] (103) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#224), dynamicpruningexpression(cs_sold_date_sk#224 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (104) ColumnarToRow [codegen id : 49] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] (105) Filter [codegen id : 49] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] +Output [2]: [cd_demo_sk#225, cd_dep_count#226] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#217] +Right keys [1]: [cd_demo_sk#225] Join condition: None (108) Project [codegen id : 49] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] +Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#227] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#224] +Right keys [1]: [d_date_sk#227] Join condition: None (111) Project [codegen id : 49] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] Batched: true 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 (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#30, ca_state#32] +Output [2]: [ca_address_sk#233, ca_state#234] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#30, ca_state#32] +Input [2]: [ca_address_sk#233, ca_state#234] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#30, ca_state#32] -Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) +Input [2]: [ca_address_sk#233, ca_state#234] +Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#30] -Input [2]: [ca_address_sk#30, ca_state#32] +Output [1]: [ca_address_sk#233] +Input [2]: [ca_address_sk#233, ca_state#234] (120) BroadcastExchange -Input [1]: [ca_address_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#183] +Input [1]: [ca_address_sk#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#230] +Right keys [1]: [ca_address_sk#233] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30] +Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] (123) BroadcastExchange -Input [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#184] +Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#185] +Output [1]: [cd_demo_sk#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#185] +Input [1]: [cd_demo_sk#237] (126) Filter -Input [1]: [cd_demo_sk#185] -Condition : isnotnull(cd_demo_sk#185) +Input [1]: [cd_demo_sk#237] +Condition : isnotnull(cd_demo_sk#237) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#185] +Left keys [1]: [c_current_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#237] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#22, c_birth_year#26] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, cd_demo_sk#185] +Output [2]: [c_customer_sk#228, c_birth_year#232] +Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] (129) BroadcastExchange -Input [2]: [c_customer_sk#22, c_birth_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#186] +Input [2]: [c_customer_sk#228, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#216] +Right keys [1]: [c_customer_sk#228] Join condition: None (131) Project [codegen id : 49] -Output [8]: [cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_customer_sk#22, c_birth_year#26] +Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#239, i_item_id#240] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#218] +Right keys [1]: [i_item_sk#239] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#19, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [10]: [cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_sk#18, i_item_id#19] +Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] +Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#19, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [1]: [i_item_id#19] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200] -Results [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] +Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#240] +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#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] +Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] (136) Exchange -Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#215] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Keys [1]: [i_item_id#19] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#216, avg(agg2#37)#217, avg(agg3#38)#218, avg(agg4#39)#219, avg(agg5#40)#220, avg(agg6#41)#221, avg(agg7#42)#222] -Results [11]: [i_item_id#19, null AS ca_country#223, null AS ca_state#224, null AS county#225, avg(agg1#36)#216 AS agg1#226, avg(agg2#37)#217 AS agg2#227, avg(agg3#38)#218 AS agg3#228, avg(agg4#39)#219 AS agg4#229, avg(agg5#40)#220 AS agg5#230, avg(agg6#41)#221 AS agg6#231, avg(agg7#42)#222 AS agg7#232] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Keys [1]: [i_item_id#240] +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)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] +Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] (138) Scan parquet default.catalog_sales -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#295), dynamicpruningexpression(cs_sold_date_sk#295 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (139) ColumnarToRow [codegen id : 57] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] (140) Filter [codegen id : 57] -Input [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] +Output [2]: [cd_demo_sk#296, cd_dep_count#297] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#288] +Right keys [1]: [cd_demo_sk#296] Join condition: None (143) Project [codegen id : 57] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] +Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#298] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#295] +Right keys [1]: [d_date_sk#298] Join condition: None (146) Project [codegen id : 57] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] (147) Scan parquet default.item -Output [1]: [i_item_sk#18] +Output [1]: [i_item_sk#299] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#18] +Input [1]: [i_item_sk#299] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#18] -Condition : isnotnull(i_item_sk#18) +Input [1]: [i_item_sk#299] +Condition : isnotnull(i_item_sk#299) (150) BroadcastExchange -Input [1]: [i_item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#233] +Input [1]: [i_item_sk#299] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#289] +Right keys [1]: [i_item_sk#299] Join condition: None (152) Project [codegen id : 57] -Output [7]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [9]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_sk#18] +Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#22, c_birth_year#26] +Output [2]: [c_customer_sk#301, c_birth_year#302] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#287] +Right keys [1]: [c_customer_sk#301] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [9]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_customer_sk#22, c_birth_year#26] +Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] +Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] (156) HashAggregate [codegen id : 57] -Input [7]: [agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] -Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] -Aggregate Attributes [14]: [sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247] -Results [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] +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#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] +Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] (157) Exchange -Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#262] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] Keys: [] -Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] -Aggregate Attributes [7]: [avg(agg1#36)#263, avg(agg2#37)#264, avg(agg3#38)#265, avg(agg4#39)#266, avg(agg5#40)#267, avg(agg6#41)#268, avg(agg7#42)#269] -Results [11]: [null AS i_item_id#270, null AS ca_country#271, null AS ca_state#272, null AS county#273, avg(agg1#36)#263 AS agg1#274, avg(agg2#37)#264 AS agg2#275, avg(agg3#38)#265 AS agg3#276, avg(agg4#39)#266 AS agg4#277, avg(agg5#40)#267 AS agg5#278, avg(agg6#41)#268 AS agg6#279, avg(agg7#42)#269 AS agg7#280] +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)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] +Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] (159) Union (160) TakeOrderedAndProject -Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] -Arguments: 100, [ca_country#33 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +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 ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (161) + + +(161) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#16] + +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 + +Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 IN dynamicpruning#10 + diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt index 4566929712713..5514e335f1b51 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt @@ -54,37 +54,37 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - 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 @@ -130,7 +130,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (24) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #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 @@ -177,7 +177,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (38) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #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 15cde1a45a99b..3c2fc95d274b6 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 @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -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_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +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#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] 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#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] 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 e66efff75180a..3f5e15881c33c 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,34 +14,34 @@ 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_sold_date_sk,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 (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - 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] #4 + 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 #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/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index 04ff822b1ce52..a993f22e559ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (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) @@ -98,7 +98,7 @@ Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnot Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -106,7 +106,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (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)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] @@ -213,95 +213,95 @@ Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 10] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Exchange -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] - -(34) Sort [codegen id : 9] -Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 - -(35) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(37) Filter [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) +(35) Filter [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(38) Project [codegen id : 10] -Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(36) Project [codegen id : 8] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(39) BroadcastExchange -Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] +(37) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_week_seq#29] +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow -Input [2]: [d_date_sk#28, d_week_seq#29] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(42) Filter -Input [2]: [d_date_sk#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#29] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None -(44) Project [codegen id : 11] -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] +(42) Project [codegen id : 9] +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] + +(43) BroadcastExchange +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] +Join condition: (d_date#16 > d_date#23 + 5 days) + +(45) Project [codegen id : 10] +Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -(45) BroadcastExchange -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] +(46) Exchange +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] -(46) Scan parquet default.inventory +(47) Sort [codegen id : 11] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(47) ColumnarToRow +(49) ColumnarToRow [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(48) Filter +(50) Filter [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [d_date_sk#28] -Right keys [1]: [inv_date_sk#31] -Join condition: None - -(50) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] - (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] -Right keys [2]: [inv_item_sk#32, d_date_sk#23] -Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) +Left keys [2]: [cs_item_sk#5, d_date_sk#27] +Right keys [2]: [inv_item_sk#32, inv_date_sk#31] +Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] 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 b88505ad7b9bc..918508787c4b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,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_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow + Exchange [cs_item_sk,d_date_sk] #3 + WholeStageCodegen (10) + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #6 + 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 #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] 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 #9 + WholeStageCodegen (9) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] 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_date_sk,inv_item_sk,inv_quantity_on_hand,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,inv_date_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,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index 6fa7b04a3b463..694b1ead8d0f0 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 @@ -36,122 +36,122 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -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_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +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#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] 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#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) Exchange -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), true, [id=#25] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#25] (28) Sort [codegen id : 10] -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 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 ded65becfe826..7066245b3973e 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,34 +17,34 @@ 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_sold_date_sk,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 (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - 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] #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] #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 #6 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date]