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 a64e8bcd6817..8b019f35263f 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 @@ -150,9 +150,16 @@ object JoinReorderDP extends PredicateHelper with Logging { // Level i maintains all found plans for i + 1 items. // Create the initial plans: each plan is a single item with zero cost. val itemIndex = items.zipWithIndex - val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map { - case (item, id) => Set(id) -> JoinPlan(Set(id), item, ExpressionSet(), Cost(0, 0)) - }.toMap) + val foundPlans = mutable.Buffer[JoinPlanMap]({ + // SPARK-32687: Change to use `LinkedHashMap` to make sure that items are + // inserted and iterated in the same order. + val joinPlanMap = new JoinPlanMap + itemIndex.foreach { + case (item, id) => + joinPlanMap.put(Set(id), JoinPlan(Set(id), item, ExpressionSet(), Cost(0, 0))) + } + joinPlanMap + }) // Build filters from the join graph to be used by the search algorithm. val filters = JoinReorderDPFilters.buildJoinGraphInfo(conf, items, conditions, itemIndex) @@ -198,7 +205,7 @@ object JoinReorderDP extends PredicateHelper with Logging { topOutput: AttributeSet, filters: Option[JoinGraphInfo]): JoinPlanMap = { - val nextLevel = mutable.Map.empty[Set[Int], JoinPlan] + val nextLevel = new JoinPlanMap var k = 0 val lev = existingLevels.length - 1 // Build plans for the next level from plans at level k (one side of the join) and level @@ -231,7 +238,7 @@ object JoinReorderDP extends PredicateHelper with Logging { } k += 1 } - nextLevel.toMap + nextLevel } /** @@ -316,7 +323,7 @@ object JoinReorderDP extends PredicateHelper with Logging { } /** Map[set of item ids, join plan for these items] */ - type JoinPlanMap = Map[Set[Int], JoinPlan] + type JoinPlanMap = mutable.LinkedHashMap[Set[Int], JoinPlan] /** * Partial join order in a specific level. 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 d9cf629b47c1..703be48c6a2a 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 @@ -345,8 +345,8 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs val expected = f1.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk"))) - .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) + .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) .join(t4.join(t3, Inner, Some(nameToAttr("t3_c2") === nameToAttr("t4_c2"))), Inner, Some(nameToAttr("d1_c2") === nameToAttr("t3_c1"))) .join(t2.join(t1, Inner, Some(nameToAttr("t1_c2") === nameToAttr("t2_c2"))), Inner, diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt index fa0104235014..b3b11b60ded0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt @@ -11,15 +11,15 @@ TakeOrderedAndProject (77) : : :- * 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.date_dim (4) : : : +- BroadcastExchange (15) : : : +- * Project (14) : : : +- * Filter (13) @@ -43,11 +43,11 @@ TakeOrderedAndProject (77) : : :- * Project (45) : : : +- * BroadcastHashJoin Inner BuildRight (44) : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildLeft (37) - : : : : :- ReusedExchange (33) - : : : : +- * Filter (36) - : : : : +- * ColumnarToRow (35) - : : : : +- Scan parquet default.store_sales (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Filter (35) + : : : : : +- * ColumnarToRow (34) + : : : : : +- Scan parquet default.store_sales (33) + : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (43) : : : +- * Project (42) : : : +- * Filter (41) @@ -65,11 +65,11 @@ TakeOrderedAndProject (77) : :- * Project (63) : : +- * BroadcastHashJoin Inner BuildRight (62) : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : : :- ReusedExchange (55) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.store_sales (56) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store_sales (55) + : : : +- ReusedExchange (58) : : +- ReusedExchange (61) : +- ReusedExchange (64) +- BroadcastExchange (70) @@ -78,50 +78,50 @@ TakeOrderedAndProject (77) +- Scan parquet default.item (67) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_year#2] +(1) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] +(2) ColumnarToRow [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] -Condition : ((((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND (d_date_sk#1 >= 2451545)) AND (d_date_sk#1 <= 2451910)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) AND (ss_sold_date_sk#1 <= 2451910)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_year#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] -(6) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] +Condition : ((((isnotnull(d_year#10) AND (d_year#10 = 2000)) AND (d_date_sk#9 >= 2451545)) AND (d_date_sk#9 <= 2451910)) AND isnotnull(d_date_sk#9)) -(7) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#9] +Input [2]: [d_date_sk#9, d_year#10] -(8) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2451545)) AND (ss_sold_date_sk#4 <= 2451910)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_item_sk#5)) +(8) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (10) Project [codegen id : 5] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (11) Scan parquet default.customer_demographics Output [4]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15] @@ -146,13 +146,13 @@ Input [1]: [cd_demo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 5] -Output [6]: [ss_item_sk#5, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#12] +Output [6]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#12] (18) Scan parquet default.store Output [2]: [s_store_sk#17, s_state#18] @@ -173,13 +173,13 @@ Input [2]: [s_store_sk#17, s_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] (22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#7] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (23) Project [codegen id : 5] -Output [6]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_state#18] -Input [8]: [ss_item_sk#5, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_store_sk#17, s_state#18] +Output [6]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_state#18] +Input [8]: [ss_item_sk#2, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17, s_state#18] (24) Scan parquet default.item Output [2]: [i_item_sk#20, i_item_id#21] @@ -200,13 +200,13 @@ Input [2]: [i_item_sk#20, i_item_id#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#20] Join condition: None (29) Project [codegen id : 5] -Output [6]: [i_item_id#21, s_state#18, ss_quantity#8 AS agg1#23, ss_list_price#9 AS agg2#24, ss_coupon_amt#11 AS agg3#25, ss_sales_price#10 AS agg4#26] -Input [8]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_state#18, i_item_sk#20, i_item_id#21] +Output [6]: [i_item_id#21, s_state#18, ss_quantity#5 AS agg1#23, ss_list_price#6 AS agg2#24, ss_coupon_amt#8 AS agg3#25, ss_sales_price#7 AS agg4#26] +Input [8]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_state#18, i_item_sk#20, i_item_id#21] (30) HashAggregate [codegen id : 5] Input [6]: [i_item_id#21, s_state#18, agg1#23, agg2#24, agg3#25, agg4#26] @@ -226,31 +226,31 @@ Functions [4]: [avg(cast(agg1#23 as bigint)), avg(UnscaledValue(agg2#24)), avg(U Aggregate Attributes [4]: [avg(cast(agg1#23 as bigint))#44, avg(UnscaledValue(agg2#24))#45, avg(UnscaledValue(agg3#25))#46, avg(UnscaledValue(agg4#26))#47] Results [7]: [i_item_id#21, s_state#18, 0 AS g_state#48, avg(cast(agg1#23 as bigint))#44 AS agg1#49, cast((avg(UnscaledValue(agg2#24))#45 / 100.0) as decimal(11,6)) AS agg2#50, cast((avg(UnscaledValue(agg3#25))#46 / 100.0) as decimal(11,6)) AS agg3#51, cast((avg(UnscaledValue(agg4#26))#47 / 100.0) as decimal(11,6)) AS agg4#52] -(33) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#1] - -(34) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(33) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(35) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(34) ColumnarToRow [codegen id : 11] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] + +(35) Filter [codegen id : 11] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) AND (ss_sold_date_sk#1 <= 2451910)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) -(36) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2451545)) AND (ss_sold_date_sk#4 <= 2451910)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_item_sk#5)) +(36) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#9] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (39) Scan parquet default.store Output [2]: [s_store_sk#17, s_state#18] @@ -275,37 +275,37 @@ Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#7] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_store_sk#17] +Output [6]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17] (46) ReusedExchange [Reuses operator id: 15] Output [1]: [cd_demo_sk#12] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#12] +Output [5]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#12] (49) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#20, i_item_id#21] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#20] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#21, ss_quantity#8 AS agg1#23, ss_list_price#9 AS agg2#24, ss_coupon_amt#11 AS agg3#25, ss_sales_price#10 AS agg4#26] -Input [7]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_sk#20, i_item_id#21] +Output [5]: [i_item_id#21, ss_quantity#5 AS agg1#23, ss_list_price#6 AS agg2#24, ss_coupon_amt#8 AS agg3#25, ss_sales_price#7 AS agg4#26] +Input [7]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#20, i_item_id#21] (52) HashAggregate [codegen id : 11] Input [5]: [i_item_id#21, agg1#23, agg2#24, agg3#25, agg4#26] @@ -325,55 +325,55 @@ Functions [4]: [avg(cast(agg1#23 as bigint)), avg(UnscaledValue(agg2#24)), avg(U Aggregate Attributes [4]: [avg(cast(agg1#23 as bigint))#71, avg(UnscaledValue(agg2#24))#72, avg(UnscaledValue(agg3#25))#73, avg(UnscaledValue(agg4#26))#74] Results [7]: [i_item_id#21, null AS s_state#75, 1 AS g_state#76, avg(cast(agg1#23 as bigint))#71 AS agg1#77, cast((avg(UnscaledValue(agg2#24))#72 / 100.0) as decimal(11,6)) AS agg2#78, cast((avg(UnscaledValue(agg3#25))#73 / 100.0) as decimal(11,6)) AS agg3#79, cast((avg(UnscaledValue(agg4#26))#74 / 100.0) as decimal(11,6)) AS agg4#80] -(55) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#1] - -(56) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(55) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2451545), LessThanOrEqual(ss_sold_date_sk,2451910), IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(57) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(56) ColumnarToRow [codegen id : 17] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] + +(57) Filter [codegen id : 17] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) AND (ss_sold_date_sk#1 <= 2451910)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_item_sk#2)) -(58) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2451545)) AND (ss_sold_date_sk#4 <= 2451910)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_store_sk#7)) AND isnotnull(ss_item_sk#5)) +(58) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#9] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (61) ReusedExchange [Reuses operator id: 43] Output [1]: [s_store_sk#17] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#7] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_store_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, s_store_sk#17] +Output [6]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, s_store_sk#17] (64) ReusedExchange [Reuses operator id: 15] Output [1]: [cd_demo_sk#12] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#12] +Output [5]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#12] (67) Scan parquet default.item Output [1]: [i_item_sk#20] @@ -394,13 +394,13 @@ Input [1]: [i_item_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#20] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#8 AS agg1#23, ss_list_price#9 AS agg2#24, ss_coupon_amt#11 AS agg3#25, ss_sales_price#10 AS agg4#26] -Input [6]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_sk#20] +Output [4]: [ss_quantity#5 AS agg1#23, ss_list_price#6 AS agg2#24, ss_coupon_amt#8 AS agg3#25, ss_sales_price#7 AS agg4#26] +Input [6]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#20] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#23, agg2#24, agg3#25, agg4#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt index fc7202e739bc..d14061de1d1f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt @@ -13,7 +13,11 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -22,10 +26,6 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -61,13 +61,13 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -93,13 +93,13 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + InputAdapter + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [s_store_sk] #7 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index 6071139e809c..220d661fd45e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject (34) : :- * 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.date_dim (4) : : +- BroadcastExchange (15) : : +- * Project (14) : : +- * Filter (13) @@ -35,50 +35,50 @@ TakeOrderedAndProject (34) +- Scan parquet default.item (25) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_year#2] +(1) Scan parquet default.store_sales +Output [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2450815), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2450815), LessThanOrEqual(ss_sold_date_sk,2451179), IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] +(2) ColumnarToRow [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] -Condition : ((((isnotnull(d_year#2) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2450815)) AND (d_date_sk#1 <= 2451179)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2450815)) AND (ss_sold_date_sk#1 <= 2451179)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_promo_sk#4)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_year#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2450815), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] -(6) Scan parquet default.store_sales -Output [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), GreaterThanOrEqual(ss_sold_date_sk,2450815), LessThanOrEqual(ss_sold_date_sk,2451179), IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] +Condition : ((((isnotnull(d_year#10) AND (d_year#10 = 1998)) AND (d_date_sk#9 >= 2450815)) AND (d_date_sk#9 <= 2451179)) AND isnotnull(d_date_sk#9)) -(7) ColumnarToRow -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#9] +Input [2]: [d_date_sk#9, d_year#10] -(8) Filter -Input [8]: [ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Condition : (((((isnotnull(ss_sold_date_sk#4) AND (ss_sold_date_sk#4 >= 2450815)) AND (ss_sold_date_sk#4 <= 2451179)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_item_sk#5)) AND isnotnull(ss_promo_sk#7)) +(8) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (10) Project [codegen id : 5] -Output [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] +Output [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, d_date_sk#9] (11) Scan parquet default.promotion Output [3]: [p_promo_sk#12, p_channel_email#13, p_channel_event#14] @@ -103,13 +103,13 @@ Input [1]: [p_promo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#7] +Left keys [1]: [ss_promo_sk#4] Right keys [1]: [p_promo_sk#12] Join condition: None (17) Project [codegen id : 5] -Output [6]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, p_promo_sk#12] +Output [6]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [8]: [ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, p_promo_sk#12] (18) Scan parquet default.customer_demographics Output [4]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19] @@ -134,13 +134,13 @@ Input [1]: [cd_demo_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#6] +Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#16] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11] -Input [7]: [ss_item_sk#5, ss_cdemo_sk#6, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, cd_demo_sk#16] +Output [5]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8] +Input [7]: [ss_item_sk#2, ss_cdemo_sk#3, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, cd_demo_sk#16] (25) Scan parquet default.item Output [2]: [i_item_sk#21, i_item_id#22] @@ -161,18 +161,18 @@ Input [2]: [i_item_sk#21, i_item_id#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#5] +Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#21] Join condition: None (30) Project [codegen id : 5] -Output [5]: [ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_id#22] -Input [7]: [ss_item_sk#5, ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_sk#21, i_item_id#22] +Output [5]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#22] +Input [7]: [ss_item_sk#2, ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_sk#21, i_item_id#22] (31) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#8, ss_list_price#9, ss_sales_price#10, ss_coupon_amt#11, i_item_id#22] +Input [5]: [ss_quantity#5, ss_list_price#6, ss_sales_price#7, ss_coupon_amt#8, i_item_id#22] Keys [1]: [i_item_id#22] -Functions [4]: [partial_avg(cast(ss_quantity#8 as bigint)), partial_avg(UnscaledValue(ss_list_price#9)), partial_avg(UnscaledValue(ss_coupon_amt#11)), partial_avg(UnscaledValue(ss_sales_price#10))] +Functions [4]: [partial_avg(cast(ss_quantity#5 as bigint)), partial_avg(UnscaledValue(ss_list_price#6)), partial_avg(UnscaledValue(ss_coupon_amt#8)), partial_avg(UnscaledValue(ss_sales_price#7))] Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] Results [9]: [i_item_id#22, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] @@ -183,9 +183,9 @@ Arguments: hashpartitioning(i_item_id#22, 5), true, [id=#40] (33) HashAggregate [codegen id : 6] Input [9]: [i_item_id#22, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Keys [1]: [i_item_id#22] -Functions [4]: [avg(cast(ss_quantity#8 as bigint)), avg(UnscaledValue(ss_list_price#9)), avg(UnscaledValue(ss_coupon_amt#11)), avg(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#8 as bigint))#41, avg(UnscaledValue(ss_list_price#9))#42, avg(UnscaledValue(ss_coupon_amt#11))#43, avg(UnscaledValue(ss_sales_price#10))#44] -Results [5]: [i_item_id#22, avg(cast(ss_quantity#8 as bigint))#41 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#9))#42 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#11))#43 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#10))#44 / 100.0) as decimal(11,6)) AS agg4#48] +Functions [4]: [avg(cast(ss_quantity#5 as bigint)), avg(UnscaledValue(ss_list_price#6)), avg(UnscaledValue(ss_coupon_amt#8)), avg(UnscaledValue(ss_sales_price#7))] +Aggregate Attributes [4]: [avg(cast(ss_quantity#5 as bigint))#41, avg(UnscaledValue(ss_list_price#6))#42, avg(UnscaledValue(ss_coupon_amt#8))#43, avg(UnscaledValue(ss_sales_price#7))#44] +Results [5]: [i_item_id#22, avg(cast(ss_quantity#5 as bigint))#41 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#6))#42 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#8))#43 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#7))#44 / 100.0) as decimal(11,6)) AS agg4#48] (34) TakeOrderedAndProject Input [5]: [i_item_id#22, agg1#45, agg2#46, agg3#47, agg4#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt index 4576b8cef59e..61cc7daa7645 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt @@ -12,7 +12,11 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [ss_promo_sk,p_promo_sk] Project [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Filter [ss_sold_date_sk,ss_cdemo_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) 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 586abbd8f3fe..8ee427262b33 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 @@ -11,14 +11,14 @@ : : :- * Project (15) : : : +- * BroadcastHashJoin Inner BuildRight (14) : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildLeft (8) - : : : : :- BroadcastExchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer_demographics (1) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet default.store_sales (5) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.customer_demographics (4) : : : +- BroadcastExchange (13) : : : +- * Filter (12) : : : +- * ColumnarToRow (11) @@ -39,46 +39,46 @@ +- Scan parquet default.customer_address (29) -(1) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +(1) Scan parquet default.store_sales +Output [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, 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] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] +ReadSchema: struct -(3) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Condition : (isnotnull(cd_demo_sk#1) AND ((((cd_marital_status#2 = M) AND (cd_education_status#3 = Advanced Degree)) OR ((cd_marital_status#2 = S) AND (cd_education_status#3 = College))) OR ((cd_marital_status#2 = W) AND (cd_education_status#3 = 2 yr Degree)))) +(2) ColumnarToRow [codegen id : 6] +Input [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, 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] -(4) BroadcastExchange -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#4] +(3) Filter [codegen id : 6] +Input [10]: [ss_sold_date_sk#1, ss_cdemo_sk#2, 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] +Condition : ((((((isnotnull(ss_store_sk#5) AND isnotnull(ss_addr_sk#4)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_cdemo_sk#2)) AND isnotnull(ss_hdemo_sk#3)) AND ((((ss_net_profit#10 >= 100.00) AND (ss_net_profit#10 <= 200.00)) OR ((ss_net_profit#10 >= 150.00) AND (ss_net_profit#10 <= 300.00))) OR ((ss_net_profit#10 >= 50.00) AND (ss_net_profit#10 <= 250.00)))) AND ((((ss_sales_price#7 >= 100.00) AND (ss_sales_price#7 <= 150.00)) OR ((ss_sales_price#7 >= 50.00) AND (ss_sales_price#7 <= 100.00))) OR ((ss_sales_price#7 >= 150.00) AND (ss_sales_price#7 <= 200.00)))) -(5) Scan parquet default.store_sales -Output [10]: [ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -(6) ColumnarToRow -Input [10]: [ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] +(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)))) -(7) Filter -Input [10]: [ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Condition : ((((((isnotnull(ss_store_sk#9) AND isnotnull(ss_addr_sk#8)) AND isnotnull(ss_sold_date_sk#5)) AND isnotnull(ss_cdemo_sk#6)) AND isnotnull(ss_hdemo_sk#7)) AND ((((ss_net_profit#14 >= 100.00) AND (ss_net_profit#14 <= 200.00)) OR ((ss_net_profit#14 >= 150.00) AND (ss_net_profit#14 <= 300.00))) OR ((ss_net_profit#14 >= 50.00) AND (ss_net_profit#14 <= 250.00)))) AND ((((ss_sales_price#11 >= 100.00) AND (ss_sales_price#11 <= 150.00)) OR ((ss_sales_price#11 >= 50.00) AND (ss_sales_price#11 <= 100.00))) OR ((ss_sales_price#11 >= 150.00) AND (ss_sales_price#11 <= 200.00)))) +(7) BroadcastExchange +Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cd_demo_sk#1] -Right keys [1]: [ss_cdemo_sk#6] -Join condition: ((((((cd_marital_status#2 = M) AND (cd_education_status#3 = Advanced Degree)) AND (ss_sales_price#11 >= 100.00)) AND (ss_sales_price#11 <= 150.00)) OR ((((cd_marital_status#2 = S) AND (cd_education_status#3 = College)) AND (ss_sales_price#11 >= 50.00)) AND (ss_sales_price#11 <= 100.00))) OR ((((cd_marital_status#2 = W) AND (cd_education_status#3 = 2 yr Degree)) AND (ss_sales_price#11 >= 150.00)) AND (ss_sales_price#11 <= 200.00))) +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))) (9) Project [codegen id : 6] -Output [11]: [cd_marital_status#2, cd_education_status#3, ss_sold_date_sk#5, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [13]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3, ss_sold_date_sk#5, ss_cdemo_sk#6, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] +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] +Input [13]: [ss_sold_date_sk#1, ss_cdemo_sk#2, 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_demo_sk#11, cd_marital_status#12, cd_education_status#13] (10) Scan parquet default.household_demographics Output [2]: [hd_demo_sk#15, hd_dep_count#16] @@ -99,13 +99,13 @@ Input [2]: [hd_demo_sk#15, hd_dep_count#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#7] +Left keys [1]: [ss_hdemo_sk#3] Right keys [1]: [hd_demo_sk#15] -Join condition: (((((((cd_marital_status#2 = M) AND (cd_education_status#3 = Advanced Degree)) AND (ss_sales_price#11 >= 100.00)) AND (ss_sales_price#11 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#2 = S) AND (cd_education_status#3 = College)) AND (ss_sales_price#11 >= 50.00)) AND (ss_sales_price#11 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#2 = W) AND (cd_education_status#3 = 2 yr Degree)) AND (ss_sales_price#11 >= 150.00)) AND (ss_sales_price#11 <= 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#5, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [13]: [cd_marital_status#2, cd_education_status#3, ss_sold_date_sk#5, ss_hdemo_sk#7, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_sales_price#11, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, hd_demo_sk#15, hd_dep_count#16] +Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] (16) Scan parquet default.date_dim Output [2]: [d_date_sk#18, d_year#19] @@ -130,13 +130,13 @@ 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_sold_date_sk#5] +Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#18] Join condition: None (22) Project [codegen id : 6] -Output [6]: [ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [8]: [ss_sold_date_sk#5, ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, d_date_sk#18] +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.store Output [1]: [s_store_sk#21] @@ -157,13 +157,13 @@ Input [1]: [s_store_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#9] +Left keys [1]: [ss_store_sk#5] Right keys [1]: [s_store_sk#21] Join condition: None (28) Project [codegen id : 6] -Output [5]: [ss_addr_sk#8, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14] -Input [7]: [ss_addr_sk#8, ss_store_sk#9, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, s_store_sk#21] +Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] +Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] (29) Scan parquet default.customer_address Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] @@ -188,18 +188,18 @@ 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_addr_sk#8] +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#14 >= 100.00)) AND (ss_net_profit#14 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#14 >= 150.00)) AND (ss_net_profit#14 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#14 >= 50.00)) AND (ss_net_profit#14 <= 250.00))) +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#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13] -Input [7]: [ss_addr_sk#8, ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13, ss_net_profit#14, ca_address_sk#23, ca_state#24] +Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] (36) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#10, ss_ext_sales_price#12, ss_ext_wholesale_cost#13] +Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] Keys: [] -Functions [4]: [partial_avg(cast(ss_quantity#10 as bigint)), partial_avg(UnscaledValue(ss_ext_sales_price#12)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#13)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#13))] +Functions [4]: [partial_avg(cast(ss_quantity#6 as bigint)), partial_avg(UnscaledValue(ss_ext_sales_price#8)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#9)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#9))] Aggregate Attributes [7]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33] Results [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] @@ -210,7 +210,7 @@ Arguments: SinglePartition, true, [id=#41] (38) HashAggregate [codegen id : 7] Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] Keys: [] -Functions [4]: [avg(cast(ss_quantity#10 as bigint)), avg(UnscaledValue(ss_ext_sales_price#12)), avg(UnscaledValue(ss_ext_wholesale_cost#13)), sum(UnscaledValue(ss_ext_wholesale_cost#13))] -Aggregate Attributes [4]: [avg(cast(ss_quantity#10 as bigint))#42, avg(UnscaledValue(ss_ext_sales_price#12))#43, avg(UnscaledValue(ss_ext_wholesale_cost#13))#44, sum(UnscaledValue(ss_ext_wholesale_cost#13))#45] -Results [4]: [avg(cast(ss_quantity#10 as bigint))#42 AS avg(ss_quantity)#46, cast((avg(UnscaledValue(ss_ext_sales_price#12))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#47, cast((avg(UnscaledValue(ss_ext_wholesale_cost#13))#44 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#48, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#13))#45,17,2) AS sum(ss_ext_wholesale_cost)#49] +Functions [4]: [avg(cast(ss_quantity#6 as bigint)), avg(UnscaledValue(ss_ext_sales_price#8)), avg(UnscaledValue(ss_ext_wholesale_cost#9)), sum(UnscaledValue(ss_ext_wholesale_cost#9))] +Aggregate Attributes [4]: [avg(cast(ss_quantity#6 as bigint))#42, avg(UnscaledValue(ss_ext_sales_price#8))#43, avg(UnscaledValue(ss_ext_wholesale_cost#9))#44, sum(UnscaledValue(ss_ext_wholesale_cost#9))#45] +Results [4]: [avg(cast(ss_quantity#6 as bigint))#42 AS avg(ss_quantity)#46, cast((avg(UnscaledValue(ss_ext_sales_price#8))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#47, cast((avg(UnscaledValue(ss_ext_wholesale_cost#9))#44 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#48, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#9))#45,17,2) AS sum(ss_ext_wholesale_cost)#49] 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 e410b27e9cf3..b457788dbd0b 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 @@ -12,8 +12,12 @@ WholeStageCodegen (7) 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 [cd_marital_status,cd_education_status,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] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk,cd_marital_status,cd_education_status,ss_sales_price] + 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] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index e24b656e843a..a17356ae04a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -39,15 +39,15 @@ TakeOrderedAndProject (57) : +- * Sort (39) : +- Exchange (38) : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildLeft (36) - : :- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.date_dim (28) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store_returns (33) + : +- * 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) @@ -177,75 +177,75 @@ Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] +(28) Scan parquet default.store_returns +Output [5]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#19, d_quarter_name#20] +(29) ColumnarToRow [codegen id : 10] +Input [5]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -(30) Filter [codegen id : 9] -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : (d_quarter_name#20 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#19)) +(30) Filter [codegen id : 10] +Input [5]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Condition : (((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isnotnull(sr_ticket_number#22)) AND isnotnull(sr_returned_date_sk#19)) -(31) Project [codegen id : 9] -Output [1]: [d_date_sk#19] -Input [2]: [d_date_sk#19, d_quarter_name#20] +(31) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_quarter_name#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +ReadSchema: struct -(32) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +(32) ColumnarToRow [codegen id : 9] +Input [2]: [d_date_sk#24, d_quarter_name#25] -(33) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(33) Filter [codegen id : 9] +Input [2]: [d_date_sk#24, d_quarter_name#25] +Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) -(34) ColumnarToRow -Input [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] +(34) Project [codegen id : 9] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_quarter_name#25] -(35) Filter -Input [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Condition : (((isnotnull(sr_customer_sk#24) AND isnotnull(sr_item_sk#23)) AND isnotnull(sr_ticket_number#25)) AND isnotnull(sr_returned_date_sk#22)) +(35) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cast(d_date_sk#19 as bigint)] -Right keys [1]: [sr_returned_date_sk#22] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (37) Project [codegen id : 10] -Output [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Input [6]: [d_date_sk#19, sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] +Output [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Input [6]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, d_date_sk#24] (38) Exchange -Input [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Arguments: hashpartitioning(sr_customer_sk#24, sr_item_sk#23, sr_ticket_number#25, 5), true, [id=#27] +Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), true, [id=#27] (39) Sort [codegen id : 11] -Input [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Arguments: [sr_customer_sk#24 ASC NULLS FIRST, sr_item_sk#23 ASC NULLS FIRST, sr_ticket_number#25 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#24, sr_item_sk#23, sr_ticket_number#25] +Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] Join condition: None (41) Project [codegen id : 12] -Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26] -Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] +Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] +Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] (42) Exchange -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26] -Arguments: hashpartitioning(sr_customer_sk#24, sr_item_sk#23, 5), true, [id=#28] +Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), true, [id=#28] (43) Sort [codegen id : 13] -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26] -Arguments: [sr_customer_sk#24 ASC NULLS FIRST, sr_item_sk#23 ASC NULLS FIRST], false, 0 +Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST], false, 0 (44) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] @@ -261,7 +261,7 @@ Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quanti Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] Condition : ((isnotnull(cs_bill_customer_sk#30) AND isnotnull(cs_item_sk#31)) AND isnotnull(cs_sold_date_sk#29)) -(47) ReusedExchange [Reuses operator id: 32] +(47) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#33] (48) BroadcastHashJoin [codegen id : 15] @@ -282,18 +282,18 @@ Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] Arguments: [cast(cs_bill_customer_sk#30 as bigint) ASC NULLS FIRST, cast(cs_item_sk#31 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#24, sr_item_sk#23] +Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] Right keys [2]: [cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint)] Join condition: None (53) Project [codegen id : 17] -Output [6]: [ss_quantity#6, sr_return_quantity#26, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] -Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#23, sr_customer_sk#24, sr_return_quantity#26, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] (54) HashAggregate [codegen id : 17] -Input [6]: [ss_quantity#6, sr_return_quantity#26, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#26), partial_avg(cast(sr_return_quantity#26 as bigint)), partial_stddev_samp(cast(sr_return_quantity#26 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#26), avg(cast(sr_return_quantity#26 as bigint)), stddev_samp(cast(sr_return_quantity#26 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#26)#75, avg(cast(sr_return_quantity#26 as bigint))#76, stddev_samp(cast(sr_return_quantity#26 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#26)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#26 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#26 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#26 as double))#77 / avg(cast(sr_return_quantity#26 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index 216adf3588ec..bfb59441f483 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -69,7 +69,11 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) @@ -78,10 +82,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_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 0fbe0ccef6d1..88b5168f6049 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 @@ -12,15 +12,15 @@ TakeOrderedAndProject (45) : :- * Project (17) : : +- * BroadcastHashJoin Inner BuildRight (16) : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.item (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) @@ -38,58 +38,58 @@ TakeOrderedAndProject (45) : +- Exchange (29) : +- * Filter (28) : +- * ColumnarToRow (27) - : +- Scan parquet default.customer_address (26) + : +- Scan parquet default.customer (26) +- * Sort (35) +- Exchange (34) +- * Filter (33) +- * ColumnarToRow (32) - +- Scan parquet default.customer (31) + +- Scan parquet default.customer_address (31) -(1) Scan parquet default.item -Output [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] +(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}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_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 : 1] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] +(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 : 1] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] -Condition : ((isnotnull(i_manager_id#6) AND (i_manager_id#6 = 8)) AND isnotnull(i_item_sk#1)) +(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) Project [codegen id : 1] -Output [5]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, i_manager_id#6] +(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) BroadcastExchange -Input [5]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(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) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -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 +(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)) -(7) ColumnarToRow -Input [5]: [ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] +(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] -(8) Filter -Input [5]: [ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Condition : (((isnotnull(ss_sold_date_sk#8) AND isnotnull(ss_item_sk#9)) AND isnotnull(ss_customer_sk#10)) AND isnotnull(ss_store_sk#11)) +(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] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#9] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#6] Join condition: None (10) Project [codegen id : 4] -Output [8]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_sold_date_sk#8, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Input [10]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_sold_date_sk#8, ss_item_sk#9, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] +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] (11) Scan parquet default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] @@ -114,13 +114,13 @@ Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] +Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#13] Join condition: None (17) Project [codegen id : 4] -Output [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12] -Input [9]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_sold_date_sk#8, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12, d_date_sk#13] +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] (18) Scan parquet default.store Output [2]: [s_store_sk#17, s_zip#18] @@ -141,111 +141,111 @@ Input [2]: [s_store_sk#17, s_zip#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] +Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#17] Join condition: None (23) Project [codegen id : 4] -Output [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18] -Input [9]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_store_sk#11, ss_ext_sales_price#12, s_store_sk#17, s_zip#18] +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] (24) Exchange -Input [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18] -Arguments: hashpartitioning(ss_customer_sk#10, 5), true, [id=#20] +Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] +Arguments: hashpartitioning(ss_customer_sk#3, 5), true, [id=#20] (25) Sort [codegen id : 5] -Input [7]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18] -Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 +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 -(26) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_zip#22] +(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_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [2]: [ca_address_sk#21, ca_zip#22] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] (28) Filter [codegen id : 6] -Input [2]: [ca_address_sk#21, ca_zip#22] -Condition : (isnotnull(ca_address_sk#21) AND isnotnull(ca_zip#22)) +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) (29) Exchange -Input [2]: [ca_address_sk#21, ca_zip#22] -Arguments: hashpartitioning(ca_address_sk#21, 5), true, [id=#23] +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: hashpartitioning(c_current_addr_sk#22, 5), true, [id=#23] (30) Sort [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_zip#22] -Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 -(31) Scan parquet default.customer -Output [2]: [c_customer_sk#24, c_current_addr_sk#25] +(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] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] +ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] +Input [2]: [ca_address_sk#24, ca_zip#25] (33) Filter [codegen id : 8] -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) +Input [2]: [ca_address_sk#24, ca_zip#25] +Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) (34) Exchange -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -Arguments: hashpartitioning(c_current_addr_sk#25, 5), true, [id=#26] +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), true, [id=#26] (35) Sort [codegen id : 9] -Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -Arguments: [c_current_addr_sk#25 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (36) SortMergeJoin [codegen id : 10] -Left keys [1]: [ca_address_sk#21] -Right keys [1]: [c_current_addr_sk#25] +Left keys [1]: [c_current_addr_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (37) Project [codegen id : 10] -Output [2]: [ca_zip#22, c_customer_sk#24] -Input [4]: [ca_address_sk#21, ca_zip#22, c_customer_sk#24, c_current_addr_sk#25] +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] (38) Exchange -Input [2]: [ca_zip#22, c_customer_sk#24] -Arguments: hashpartitioning(c_customer_sk#24, 5), true, [id=#27] +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: hashpartitioning(c_customer_sk#21, 5), true, [id=#27] (39) Sort [codegen id : 11] -Input [2]: [ca_zip#22, c_customer_sk#24] -Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#10] -Right keys [1]: [c_customer_sk#24] -Join condition: NOT (substr(ca_zip#22, 1, 5) = substr(s_zip#18, 1, 5)) +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#12, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Input [9]: [i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5, ss_customer_sk#10, ss_ext_sales_price#12, s_zip#18, ca_zip#22, c_customer_sk#24] +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#12, i_brand_id#2, i_brand#3, i_manufact_id#4, i_manufact#5] -Keys [4]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] +Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, 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#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, sum#29] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, 5), true, [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), true, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5, sum#29] -Keys [4]: [i_brand#3, i_brand_id#2, i_manufact_id#4, i_manufact#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#12))#31] -Results [5]: [i_brand_id#2 AS brand_id#32, i_brand#3 AS brand#33, i_manufact_id#4, i_manufact#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#12))#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#4, i_manufact#5, 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#4 ASC NULLS FIRST, i_manufact#5 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#4, i_manufact#5, 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 c8737d8a7078..05fa3f82e27d 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 @@ -13,12 +13,16 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] InputAdapter Exchange [ss_customer_sk] #2 WholeStageCodegen (4) - Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_customer_sk,ss_ext_sales_price,s_zip] + 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 [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + 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 [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_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) @@ -27,10 +31,6 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -52,25 +52,25 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (10) - Project [ca_zip,c_customer_sk] - SortMergeJoin [ca_address_sk,c_current_addr_sk] + Project [c_customer_sk,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (7) - Sort [ca_address_sk] + Sort [c_current_addr_sk] InputAdapter - Exchange [ca_address_sk] #7 + Exchange [c_current_addr_sk] #7 WholeStageCodegen (6) - Filter [ca_address_sk,ca_zip] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (9) - Sort [c_current_addr_sk] + Sort [ca_address_sk] InputAdapter - Exchange [c_current_addr_sk] #8 + Exchange [ca_address_sk] #8 WholeStageCodegen (8) - Filter [c_customer_sk,c_current_addr_sk] + Filter [ca_address_sk,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + 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/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index 6e0a5ced1992..ffcf6bd4f6d4 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 @@ -296,15 +296,15 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : :- * Sort (60) : : : : +- Exchange (59) : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildLeft (57) - : : : : :- BroadcastExchange (53) - : : : : : +- * Project (52) - : : : : : +- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store (49) - : : : : +- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.store_sales (54) + : : : : +- * 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) @@ -327,57 +327,57 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer +- Scan parquet default.store_returns (88) -(49) Scan parquet default.store +(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] +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] + +(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)) + +(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] 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 -(50) ColumnarToRow [codegen id : 1] +(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] -(51) Filter [codegen id : 1] +(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)) -(52) Project [codegen id : 1] +(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] -(53) BroadcastExchange +(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] -(54) 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] -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 - -(55) ColumnarToRow -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(56) Filter -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)) - (57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#19] -Right keys [1]: [ss_store_sk#3] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#19] Join condition: None (58) Project [codegen id : 2] -Output [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] -Input [9]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +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] (59) Exchange -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] (60) Sort [codegen id : 3] -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (61) Scan parquet default.item @@ -408,15 +408,15 @@ Right keys [1]: [i_item_sk#6] Join condition: None (67) Project [codegen id : 6] -Output [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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 [13]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (68) Exchange -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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 [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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 [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (70) Scan parquet default.customer @@ -447,15 +447,15 @@ Right keys [1]: [c_customer_sk#14] Join condition: None (76) Project [codegen id : 10] -Output [14]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17] -Input [16]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (77) Exchange -Input [14]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [s_store_name#20, s_state#22, 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, ca_state#25] -Input [17]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [s_store_name#20, s_state#22, 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, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [s_store_name#20, s_state#22, 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, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [s_store_name#20, s_state#22, 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, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (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] 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 f51d1972b630..10f874f8f554 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 [s_store_name,s_state,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,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,7 +29,7 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + 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) @@ -37,7 +37,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + 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) @@ -45,8 +45,12 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_item_sk] #15 WholeStageCodegen (2) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] + 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] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) @@ -55,10 +59,6 @@ WholeStageCodegen (14) ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] 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 cbac3787cab6..73f36e3a9ca2 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 @@ -296,15 +296,15 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : :- * Sort (60) : : : : +- Exchange (59) : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildLeft (57) - : : : : :- BroadcastExchange (53) - : : : : : +- * Project (52) - : : : : : +- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store (49) - : : : : +- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.store_sales (54) + : : : : +- * 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) @@ -327,57 +327,57 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer +- Scan parquet default.store_returns (88) -(49) Scan parquet default.store +(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] +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] + +(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)) + +(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] 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 -(50) ColumnarToRow [codegen id : 1] +(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] -(51) Filter [codegen id : 1] +(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)) -(52) Project [codegen id : 1] +(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] -(53) BroadcastExchange +(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] -(54) 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] -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 - -(55) ColumnarToRow -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(56) Filter -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)) - (57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#19] -Right keys [1]: [ss_store_sk#3] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#19] Join condition: None (58) Project [codegen id : 2] -Output [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] -Input [9]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +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] (59) Exchange -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] (60) Sort [codegen id : 3] -Input [7]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (61) Scan parquet default.item @@ -408,15 +408,15 @@ Right keys [1]: [i_item_sk#6] Join condition: None (67) Project [codegen id : 6] -Output [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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 [13]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (68) Exchange -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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 [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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 [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (70) Scan parquet default.customer @@ -447,15 +447,15 @@ Right keys [1]: [c_customer_sk#14] Join condition: None (76) Project [codegen id : 10] -Output [14]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17] -Input [16]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk#2, 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] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (77) Exchange -Input [14]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [s_store_name#20, s_state#22, 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, ca_state#25] -Input [17]: [s_store_name#20, s_state#22, s_zip#23, 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, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [s_store_name#20, s_state#22, 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, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [s_store_name#20, s_state#22, 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, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [s_store_name#20, s_state#22, 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, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (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] 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 f51d1972b630..10f874f8f554 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 [s_store_name,s_state,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,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,7 +29,7 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + 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) @@ -37,7 +37,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + 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) @@ -45,8 +45,12 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_item_sk] #15 WholeStageCodegen (2) - Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] + 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] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) @@ -55,10 +59,6 @@ WholeStageCodegen (14) ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] 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 87a72d3bbe77..c6dc3db86900 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 @@ -39,15 +39,15 @@ TakeOrderedAndProject (57) : +- * Sort (39) : +- Exchange (38) : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildLeft (36) - : :- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.date_dim (28) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store_returns (33) + : +- * 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) @@ -177,75 +177,75 @@ Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] +(28) Scan parquet default.store_returns +Output [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(29) ColumnarToRow [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -(30) Filter [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 >= 4)) AND (d_moy#23 <= 10)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#21)) +(30) Filter [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Condition : (((isnotnull(sr_customer_sk#23) AND isnotnull(sr_item_sk#22)) AND isnotnull(sr_ticket_number#24)) AND isnotnull(sr_returned_date_sk#21)) -(31) Project [codegen id : 9] -Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(31) Scan parquet default.date_dim +Output [3]: [d_date_sk#26, d_year#27, d_moy#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(32) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(32) ColumnarToRow [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(33) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(33) Filter [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] +Condition : (((((isnotnull(d_moy#28) AND isnotnull(d_year#27)) AND (d_moy#28 >= 4)) AND (d_moy#28 <= 10)) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) -(34) ColumnarToRow -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] +(34) Project [codegen id : 9] +Output [1]: [d_date_sk#26] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(35) Filter -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Condition : (((isnotnull(sr_customer_sk#27) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_ticket_number#28)) AND isnotnull(sr_returned_date_sk#25)) +(35) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cast(d_date_sk#21 as bigint)] -Right keys [1]: [sr_returned_date_sk#25] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (37) Project [codegen id : 10] -Output [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Input [6]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] +Output [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, d_date_sk#26] (38) Exchange -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28, 5), true, [id=#30] +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] (39) Sort [codegen id : 11] -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_ticket_number#28 ASC NULLS FIRST], false, 0 +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#27, sr_item_sk#26, sr_ticket_number#28] +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#26, sr_customer_sk#27, sr_net_loss#29] -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#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] +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#26, sr_customer_sk#27, sr_net_loss#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, 5), true, [id=#31] +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] (43) Sort [codegen id : 13] -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_net_loss#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST], false, 0 +Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 (44) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] @@ -261,7 +261,7 @@ Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_pr 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)) -(47) ReusedExchange [Reuses operator id: 32] +(47) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#36] (48) BroadcastHashJoin [codegen id : 15] @@ -282,18 +282,18 @@ 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 [2]: [sr_customer_sk#27, sr_item_sk#26] +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#29, 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#26, sr_customer_sk#27, sr_net_loss#29, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#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#29, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#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#29)), partial_sum(UnscaledValue(cs_net_profit#35))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#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#29)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#29))#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#29))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#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 8e61cf9c519f..ad9fa718ff2b 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 @@ -69,7 +69,11 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales 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 [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) @@ -78,10 +82,6 @@ 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] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index 35e24698c517..a949b93f3bcb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -39,15 +39,15 @@ TakeOrderedAndProject (61) : +- * Sort (39) : +- Exchange (38) : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildLeft (36) - : :- BroadcastExchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.date_dim (28) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.store_returns (33) + : +- * 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 (55) +- Exchange (54) +- * Project (53) @@ -181,75 +181,75 @@ Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] +(28) Scan parquet default.store_returns +Output [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(29) ColumnarToRow [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] -(30) Filter [codegen id : 9] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 >= 9)) AND (d_moy#23 <= 12)) AND (d_year#22 = 1999)) AND isnotnull(d_date_sk#21)) +(30) Filter [codegen id : 10] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Condition : (((isnotnull(sr_customer_sk#23) AND isnotnull(sr_item_sk#22)) AND isnotnull(sr_ticket_number#24)) AND isnotnull(sr_returned_date_sk#21)) -(31) Project [codegen id : 9] -Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(31) Scan parquet default.date_dim +Output [3]: [d_date_sk#26, d_year#27, d_moy#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(32) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(32) ColumnarToRow [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(33) Scan parquet default.store_returns -Output [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(33) Filter [codegen id : 9] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] +Condition : (((((isnotnull(d_moy#28) AND isnotnull(d_year#27)) AND (d_moy#28 >= 9)) AND (d_moy#28 <= 12)) AND (d_year#27 = 1999)) AND isnotnull(d_date_sk#26)) -(34) ColumnarToRow -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] +(34) Project [codegen id : 9] +Output [1]: [d_date_sk#26] +Input [3]: [d_date_sk#26, d_year#27, d_moy#28] -(35) Filter -Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Condition : (((isnotnull(sr_customer_sk#27) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_ticket_number#28)) AND isnotnull(sr_returned_date_sk#25)) +(35) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cast(d_date_sk#21 as bigint)] -Right keys [1]: [sr_returned_date_sk#25] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (37) Project [codegen id : 10] -Output [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Input [6]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] +Output [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25, d_date_sk#26] (38) Exchange -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28, 5), true, [id=#30] +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] (39) Sort [codegen id : 11] -Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_ticket_number#28 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_return_quantity#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#27, sr_item_sk#26, sr_ticket_number#28] +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_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29] -Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] +Output [8]: [ss_quantity#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_return_quantity#25] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#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_return_quantity#25] (42) Exchange -Input [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29] -Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, 5), true, [id=#31] +Input [8]: [ss_quantity#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_return_quantity#25] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] (43) Sort [codegen id : 13] -Input [8]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29] -Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST], false, 0 +Input [8]: [ss_quantity#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_return_quantity#25] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 (44) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] @@ -305,18 +305,18 @@ Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#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 (56) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#27, sr_item_sk#26] +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 (57) Project [codegen id : 17] -Output [7]: [ss_quantity#6, sr_return_quantity#29, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [11]: [ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#26, sr_customer_sk#27, sr_return_quantity#29, cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] +Output [7]: [ss_quantity#6, sr_return_quantity#25, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [11]: [ss_quantity#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_return_quantity#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_quantity#35] (58) HashAggregate [codegen id : 17] -Input [7]: [ss_quantity#6, sr_return_quantity#29, cs_quantity#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_quantity#6, sr_return_quantity#25, cs_quantity#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(cast(ss_quantity#6 as bigint)), partial_sum(cast(sr_return_quantity#29 as bigint)), partial_sum(cast(cs_quantity#35 as bigint))] +Functions [3]: [partial_sum(cast(ss_quantity#6 as bigint)), partial_sum(cast(sr_return_quantity#25 as bigint)), partial_sum(cast(cs_quantity#35 as bigint))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#43, sum#44, sum#45] @@ -327,9 +327,9 @@ Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store (60) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#43, sum#44, sum#45] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(cast(ss_quantity#6 as bigint)), sum(cast(sr_return_quantity#29 as bigint)), sum(cast(cs_quantity#35 as bigint))] -Aggregate Attributes [3]: [sum(cast(ss_quantity#6 as bigint))#47, sum(cast(sr_return_quantity#29 as bigint))#48, sum(cast(cs_quantity#35 as bigint))#49] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum(cast(ss_quantity#6 as bigint))#47 AS store_sales_quantity#50, sum(cast(sr_return_quantity#29 as bigint))#48 AS store_returns_quantity#51, sum(cast(cs_quantity#35 as bigint))#49 AS catalog_sales_quantity#52] +Functions [3]: [sum(cast(ss_quantity#6 as bigint)), sum(cast(sr_return_quantity#25 as bigint)), sum(cast(cs_quantity#35 as bigint))] +Aggregate Attributes [3]: [sum(cast(ss_quantity#6 as bigint))#47, sum(cast(sr_return_quantity#25 as bigint))#48, sum(cast(cs_quantity#35 as bigint))#49] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum(cast(ss_quantity#6 as bigint))#47 AS store_sales_quantity#50, sum(cast(sr_return_quantity#25 as bigint))#48 AS store_returns_quantity#51, sum(cast(cs_quantity#35 as bigint))#49 AS catalog_sales_quantity#52] (61) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_quantity#50, store_returns_quantity#51, catalog_sales_quantity#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt index f10b8e245c50..ea91af9e8f75 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt @@ -69,7 +69,11 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) @@ -78,10 +82,6 @@ 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] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt index d3b013660ba2..9f123c4044cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt @@ -138,7 +138,7 @@ Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) Output [3]: [d_date_sk#4, d_year#5, d_qoy#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -146,7 +146,7 @@ Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] (6) Filter [codegen id : 1] Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] -Condition : ((((isnotnull(d_qoy#6) AND isnotnull(d_year#5)) AND (d_qoy#6 = 3)) AND (d_year#5 = 2000)) AND isnotnull(d_date_sk#4)) +Condition : ((((isnotnull(d_qoy#6) AND isnotnull(d_year#5)) AND (d_qoy#6 = 2)) AND (d_year#5 = 2000)) AND isnotnull(d_date_sk#4)) (7) BroadcastExchange Input [3]: [d_date_sk#4, d_year#5, d_qoy#6] @@ -236,7 +236,7 @@ Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#2)) Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 7] @@ -244,7 +244,7 @@ Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] (27) Filter [codegen id : 7] Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) +Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 3)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) (28) BroadcastExchange Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] @@ -311,7 +311,7 @@ Right keys [1]: [ca_county#23] Join condition: None (42) Project [codegen id : 42] -Output [3]: [store_sales#16, ca_county#23, store_sales#28] +Output [3]: [ca_county#10, store_sales#16, store_sales#28] Input [4]: [ca_county#10, store_sales#16, ca_county#23, store_sales#28] (43) Scan parquet default.store_sales @@ -402,13 +402,13 @@ Input [3]: [ca_county#36, d_year#31, store_sales#41] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] (62) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#23] +Left keys [1]: [ca_county#10] Right keys [1]: [ca_county#36] Join condition: None (63) Project [codegen id : 42] Output [5]: [store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41] -Input [6]: [store_sales#16, ca_county#23, store_sales#28, ca_county#36, d_year#31, store_sales#41] +Input [6]: [ca_county#10, store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41] (64) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] @@ -424,7 +424,7 @@ Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Condition : (isnotnull(ws_sold_date_sk#43) AND isnotnull(ws_bill_addr_sk#44)) -(67) ReusedExchange [Reuses operator id: 28] +(67) ReusedExchange [Reuses operator id: 49] Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] (68) BroadcastHashJoin [codegen id : 22] @@ -492,7 +492,7 @@ Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Condition : (isnotnull(ws_sold_date_sk#43) AND isnotnull(ws_bill_addr_sk#44)) -(82) ReusedExchange [Reuses operator id: 49] +(82) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] (83) BroadcastHashJoin [codegen id : 28] @@ -556,7 +556,7 @@ Right keys [1]: [ca_county#62] Join condition: None (96) Project [codegen id : 41] -Output [3]: [web_sales#56, ca_county#62, web_sales#67] +Output [3]: [ca_county#51, web_sales#56, web_sales#67] Input [4]: [ca_county#51, web_sales#56, ca_county#62, web_sales#67] (97) Scan parquet default.web_sales @@ -573,7 +573,7 @@ Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Input [3]: [ws_sold_date_sk#43, ws_bill_addr_sk#44, ws_ext_sales_price#45] Condition : (isnotnull(ws_sold_date_sk#43) AND isnotnull(ws_bill_addr_sk#44)) -(100) ReusedExchange [Reuses operator id: 7] +(100) ReusedExchange [Reuses operator id: 28] Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] (101) BroadcastHashJoin [codegen id : 35] @@ -632,26 +632,26 @@ Input [2]: [ca_county#74, web_sales#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#62] +Left keys [1]: [ca_county#51] Right keys [1]: [ca_county#74] Join condition: None (114) Project [codegen id : 41] -Output [4]: [web_sales#56, ca_county#62, web_sales#67, web_sales#79] -Input [5]: [web_sales#56, ca_county#62, web_sales#67, ca_county#74, web_sales#79] +Output [4]: [ca_county#51, web_sales#56, web_sales#67, web_sales#79] +Input [5]: [ca_county#51, web_sales#56, web_sales#67, ca_county#74, web_sales#79] (115) BroadcastExchange -Input [4]: [web_sales#56, ca_county#62, web_sales#67, web_sales#79] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#81] +Input [4]: [ca_county#51, web_sales#56, web_sales#67, web_sales#79] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#81] (116) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#36] -Right keys [1]: [ca_county#62] -Join condition: ((CASE WHEN (web_sales#67 > 0.00) THEN CheckOverflow((promote_precision(web_sales#56) / promote_precision(web_sales#67)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#41 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#41)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#56 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#56)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END)) +Right keys [1]: [ca_county#51] +Join condition: ((CASE WHEN (web_sales#56 > 0.00) THEN CheckOverflow((promote_precision(web_sales#67) / promote_precision(web_sales#56)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#41 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#41)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#67 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#67)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#36, d_year#31, CheckOverflow((promote_precision(web_sales#56) / promote_precision(web_sales#67)), DecimalType(37,20), true) AS web_q1_q2_increase#82, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#41)), DecimalType(37,20), true) AS store_q1_q2_increase#83, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#56)), DecimalType(37,20), true) AS web_q2_q3_increase#84, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#85] -Input [9]: [store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41, web_sales#56, ca_county#62, web_sales#67, web_sales#79] +Output [6]: [ca_county#36, d_year#31, CheckOverflow((promote_precision(web_sales#67) / promote_precision(web_sales#56)), DecimalType(37,20), true) AS web_q1_q2_increase#82, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#41)), DecimalType(37,20), true) AS store_q1_q2_increase#83, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#67)), DecimalType(37,20), true) AS web_q2_q3_increase#84, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#85] +Input [9]: [store_sales#16, store_sales#28, ca_county#36, d_year#31, store_sales#41, ca_county#51, web_sales#56, web_sales#67, web_sales#79] (118) Exchange Input [6]: [ca_county#36, d_year#31, web_q1_q2_increase#82, store_q1_q2_increase#83, web_q2_q3_increase#84, store_q2_q3_increase#85] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt index 9ec06b597cb6..c7b69500ed8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt @@ -7,7 +7,7 @@ WholeStageCodegen (43) BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] Project [store_sales,store_sales,ca_county,d_year,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [store_sales,ca_county,store_sales] + Project [ca_county,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter @@ -116,9 +116,9 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #14 WholeStageCodegen (41) - Project [web_sales,ca_county,web_sales,web_sales] + Project [ca_county,web_sales,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county] - Project [web_sales,ca_county,web_sales] + Project [ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county] HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter @@ -140,7 +140,7 @@ WholeStageCodegen (43) InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 InputAdapter WholeStageCodegen (25) Sort [ca_address_sk] @@ -169,7 +169,7 @@ WholeStageCodegen (43) InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 InputAdapter WholeStageCodegen (31) Sort [ca_address_sk] @@ -198,7 +198,7 @@ WholeStageCodegen (43) InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 InputAdapter WholeStageCodegen (38) Sort [ca_address_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 0232d56ab748..54e117e6cac1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -34,12 +34,12 @@ TakeOrderedAndProject (46) : : +- Exchange (22) : : +- * Filter (21) : : +- * ColumnarToRow (20) - : : +- Scan parquet default.customer_address (19) + : : +- Scan parquet default.customer (19) : +- * Sort (28) : +- Exchange (27) : +- * Filter (26) : +- * ColumnarToRow (25) - : +- Scan parquet default.customer (24) + : +- Scan parquet default.customer_address (24) +- BroadcastExchange (39) +- * Project (38) +- * Filter (37) @@ -127,75 +127,75 @@ Arguments: hashpartitioning(ws_bill_customer_sk#4, 5), true, [id=#13] Input [3]: [ws_bill_customer_sk#4, ws_sales_price#5, i_item_id#11] Arguments: [ws_bill_customer_sk#4 ASC NULLS FIRST], false, 0 -(19) Scan parquet default.customer_address -Output [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] +(19) Scan parquet default.customer +Output [2]: [c_customer_sk#14, c_current_addr_sk#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct +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 [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] (21) Filter [codegen id : 5] -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] -Condition : isnotnull(ca_address_sk#14) +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) (22) Exchange -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] -Arguments: hashpartitioning(ca_address_sk#14, 5), true, [id=#17] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), true, [id=#16] (23) Sort [codegen id : 6] -Input [3]: [ca_address_sk#14, ca_city#15, ca_zip#16] -Arguments: [ca_address_sk#14 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +(24) Scan parquet default.customer_address +Output [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct (25) ColumnarToRow [codegen id : 7] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] (26) Filter [codegen id : 7] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] +Condition : isnotnull(ca_address_sk#17) (27) Exchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_current_addr_sk#19, 5), true, [id=#20] +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] +Arguments: hashpartitioning(ca_address_sk#17, 5), true, [id=#20] (28) Sort [codegen id : 8] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: [c_current_addr_sk#19 ASC NULLS FIRST], false, 0 +Input [3]: [ca_address_sk#17, ca_city#18, ca_zip#19] +Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 (29) SortMergeJoin [codegen id : 9] -Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#19] +Left keys [1]: [c_current_addr_sk#15] +Right keys [1]: [ca_address_sk#17] Join condition: None (30) Project [codegen id : 9] -Output [3]: [ca_city#15, ca_zip#16, c_customer_sk#18] -Input [5]: [ca_address_sk#14, ca_city#15, ca_zip#16, c_customer_sk#18, c_current_addr_sk#19] +Output [3]: [c_customer_sk#14, ca_city#18, ca_zip#19] +Input [5]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_city#18, ca_zip#19] (31) Exchange -Input [3]: [ca_city#15, ca_zip#16, c_customer_sk#18] -Arguments: hashpartitioning(c_customer_sk#18, 5), true, [id=#21] +Input [3]: [c_customer_sk#14, ca_city#18, ca_zip#19] +Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#21] (32) Sort [codegen id : 10] -Input [3]: [ca_city#15, ca_zip#16, c_customer_sk#18] -Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#14, ca_city#18, ca_zip#19] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 12] Left keys [1]: [ws_bill_customer_sk#4] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#14] Join condition: None (34) Project [codegen id : 12] -Output [4]: [ws_sales_price#5, ca_city#15, ca_zip#16, i_item_id#11] -Input [6]: [ws_bill_customer_sk#4, ws_sales_price#5, i_item_id#11, ca_city#15, ca_zip#16, c_customer_sk#18] +Output [4]: [ws_sales_price#5, ca_city#18, ca_zip#19, i_item_id#11] +Input [6]: [ws_bill_customer_sk#4, ws_sales_price#5, i_item_id#11, c_customer_sk#14, ca_city#18, ca_zip#19] (35) Scan parquet default.item Output [2]: [i_item_sk#10, i_item_id#11] @@ -225,32 +225,32 @@ Right keys [1]: [i_item_id#11#22] Join condition: None (41) Filter [codegen id : 12] -Input [5]: [ws_sales_price#5, ca_city#15, ca_zip#16, i_item_id#11, exists#1] -Condition : (substr(ca_zip#16, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) +Input [5]: [ws_sales_price#5, ca_city#18, ca_zip#19, i_item_id#11, exists#1] +Condition : (substr(ca_zip#19, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) (42) Project [codegen id : 12] -Output [3]: [ws_sales_price#5, ca_city#15, ca_zip#16] -Input [5]: [ws_sales_price#5, ca_city#15, ca_zip#16, i_item_id#11, exists#1] +Output [3]: [ws_sales_price#5, ca_city#18, ca_zip#19] +Input [5]: [ws_sales_price#5, ca_city#18, ca_zip#19, i_item_id#11, exists#1] (43) HashAggregate [codegen id : 12] -Input [3]: [ws_sales_price#5, ca_city#15, ca_zip#16] -Keys [2]: [ca_zip#16, ca_city#15] +Input [3]: [ws_sales_price#5, ca_city#18, ca_zip#19] +Keys [2]: [ca_zip#19, ca_city#18] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#5))] Aggregate Attributes [1]: [sum#24] -Results [3]: [ca_zip#16, ca_city#15, sum#25] +Results [3]: [ca_zip#19, ca_city#18, sum#25] (44) Exchange -Input [3]: [ca_zip#16, ca_city#15, sum#25] -Arguments: hashpartitioning(ca_zip#16, ca_city#15, 5), true, [id=#26] +Input [3]: [ca_zip#19, ca_city#18, sum#25] +Arguments: hashpartitioning(ca_zip#19, ca_city#18, 5), true, [id=#26] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#16, ca_city#15, sum#25] -Keys [2]: [ca_zip#16, ca_city#15] +Input [3]: [ca_zip#19, ca_city#18, sum#25] +Keys [2]: [ca_zip#19, ca_city#18] Functions [1]: [sum(UnscaledValue(ws_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#5))#27] -Results [3]: [ca_zip#16, ca_city#15, MakeDecimal(sum(UnscaledValue(ws_sales_price#5))#27,17,2) AS sum(ws_sales_price)#28] +Results [3]: [ca_zip#19, ca_city#18, MakeDecimal(sum(UnscaledValue(ws_sales_price#5))#27,17,2) AS sum(ws_sales_price)#28] (46) TakeOrderedAndProject -Input [3]: [ca_zip#16, ca_city#15, sum(ws_sales_price)#28] -Arguments: 100, [ca_zip#16 ASC NULLS FIRST, ca_city#15 ASC NULLS FIRST], [ca_zip#16, ca_city#15, sum(ws_sales_price)#28] +Input [3]: [ca_zip#19, ca_city#18, sum(ws_sales_price)#28] +Arguments: 100, [ca_zip#19 ASC NULLS FIRST, ca_city#18 ASC NULLS FIRST], [ca_zip#19, ca_city#18, sum(ws_sales_price)#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt index 1eab468e67bc..0e9662bb6aca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt @@ -45,28 +45,28 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (9) - Project [ca_city,ca_zip,c_customer_sk] - SortMergeJoin [ca_address_sk,c_current_addr_sk] + Project [c_customer_sk,ca_city,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (6) - Sort [ca_address_sk] + Sort [c_current_addr_sk] InputAdapter - Exchange [ca_address_sk] #6 + Exchange [c_current_addr_sk] #6 WholeStageCodegen (5) - Filter [ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_zip] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (8) - Sort [c_current_addr_sk] + Sort [ca_address_sk] InputAdapter - Exchange [c_current_addr_sk] #7 + Exchange [ca_address_sk] #7 WholeStageCodegen (7) - Filter [c_customer_sk,c_current_addr_sk] + Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + Scan parquet default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt index df1197d7c925..741ee50f800e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject (35) +- * Sort (29) +- Exchange (28) +- * Project (27) - +- * BroadcastHashJoin Inner BuildLeft (26) - :- BroadcastExchange (22) - : +- * Project (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.date_dim (18) - +- * Filter (25) - +- * ColumnarToRow (24) - +- Scan parquet default.store_returns (23) + +- * BroadcastHashJoin Inner BuildRight (26) + :- * Filter (20) + : +- * ColumnarToRow (19) + : +- Scan parquet default.store_returns (18) + +- BroadcastExchange (25) + +- * Project (24) + +- * Filter (23) + +- * ColumnarToRow (22) + +- Scan parquet default.date_dim (21) (1) Scan parquet default.store_sales @@ -112,72 +112,72 @@ Arguments: hashpartitioning(cast(ss_ticket_number#5 as bigint), cast(ss_item_sk# Input [14]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] Arguments: [cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST], false, 0 -(18) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] +(18) Scan parquet default.store_returns +Output [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] +ReadSchema: struct -(19) ColumnarToRow [codegen id : 5] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(19) ColumnarToRow [codegen id : 6] +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] -(20) Filter [codegen id : 5] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : ((((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 2001)) AND (d_moy#23 = 8)) AND isnotnull(d_date_sk#21)) +(20) Filter [codegen id : 6] +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Condition : (((isnotnull(sr_ticket_number#24) AND isnotnull(sr_item_sk#22)) AND isnotnull(sr_customer_sk#23)) AND isnotnull(sr_returned_date_sk#21)) -(21) Project [codegen id : 5] -Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] +(21) Scan parquet default.date_dim +Output [3]: [d_date_sk#25, d_year#26, d_moy#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] +ReadSchema: struct -(22) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(22) ColumnarToRow [codegen id : 5] +Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -(23) Scan parquet default.store_returns -Output [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] -ReadSchema: struct +(23) Filter [codegen id : 5] +Input [3]: [d_date_sk#25, d_year#26, d_moy#27] +Condition : ((((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2001)) AND (d_moy#27 = 8)) AND isnotnull(d_date_sk#25)) -(24) ColumnarToRow -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] +(24) Project [codegen id : 5] +Output [1]: [d_date_sk#25] +Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -(25) Filter -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Condition : (((isnotnull(sr_ticket_number#28) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_customer_sk#27)) AND isnotnull(sr_returned_date_sk#25)) +(25) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cast(d_date_sk#21 as bigint)] -Right keys [1]: [sr_returned_date_sk#25] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (27) Project [codegen id : 6] -Output [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Input [5]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] +Output [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Input [5]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, d_date_sk#25] (28) Exchange -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Arguments: hashpartitioning(sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27, 5), true, [id=#29] +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Arguments: hashpartitioning(sr_ticket_number#24, sr_item_sk#22, sr_customer_sk#23, 5), true, [id=#29] (29) Sort [codegen id : 7] -Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Arguments: [sr_ticket_number#28 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [4]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] +Arguments: [sr_ticket_number#24 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_customer_sk#23 ASC NULLS FIRST], false, 0 (30) SortMergeJoin [codegen id : 8] Left keys [3]: [cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint)] -Right keys [3]: [sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27] +Right keys [3]: [sr_ticket_number#24, sr_item_sk#22, sr_customer_sk#23] Join condition: None (31) Project [codegen id : 8] -Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#25, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] -Input [18]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] +Output [12]: [ss_sold_date_sk#1, sr_returned_date_sk#21, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] +Input [18]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24] (32) HashAggregate [codegen id : 8] -Input [12]: [ss_sold_date_sk#1, sr_returned_date_sk#25, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] +Input [12]: [ss_sold_date_sk#1, sr_returned_date_sk#21, s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] Keys [10]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] -Functions [5]: [partial_sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] +Functions [5]: [partial_sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] Aggregate Attributes [5]: [sum#30, sum#31, sum#32, sum#33, sum#34] Results [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum#35, sum#36, sum#37, sum#38, sum#39] @@ -188,9 +188,9 @@ Arguments: hashpartitioning(s_store_name#9, s_company_id#10, s_street_number#11, (34) HashAggregate [codegen id : 9] Input [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum#35, sum#36, sum#37, sum#38, sum#39] Keys [10]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18] -Functions [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44, sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45] -Results [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41 AS 30 days #46, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42 AS 31 - 60 days #47, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43 AS 61 - 90 days #48, sum(cast(CASE WHEN (((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44 AS 91 - 120 days #49, sum(cast(CASE WHEN ((sr_returned_date_sk#25 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45 AS >120 days #50] +Functions [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] +Aggregate Attributes [5]: [sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44, sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45] +Results [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 30) THEN 1 ELSE 0 END as bigint))#41 AS 30 days #46, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 30) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint))#42 AS 31 - 60 days #47, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 60) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint))#43 AS 61 - 90 days #48, sum(cast(CASE WHEN (((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 90) AND ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint))#44 AS 91 - 120 days #49, sum(cast(CASE WHEN ((sr_returned_date_sk#21 - cast(ss_sold_date_sk#1 as bigint)) > 120) THEN 1 ELSE 0 END as bigint))#45 AS >120 days #50] (35) TakeOrderedAndProject Input [15]: [s_store_name#9, s_company_id#10, s_street_number#11, s_street_name#12, s_street_type#13, s_suite_number#14, s_city#15, s_county#16, s_state#17, s_zip#18, 30 days #46, 31 - 60 days #47, 61 - 90 days #48, 91 - 120 days #49, >120 days #50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt index 5d6f38e882a5..be11a6917681 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt @@ -42,7 +42,11 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s Exchange [sr_ticket_number,sr_item_sk,sr_customer_sk] #5 WholeStageCodegen (6) Project [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) @@ -51,7 +55,3 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 511e1b46cd7a..675cff99ad72 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -11,30 +11,30 @@ TakeOrderedAndProject (50) : +- Exchange (27) : +- * Project (26) : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) : : :- * 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 (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * BroadcastHashJoin LeftOuter BuildRight (14) + : : :- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) : +- BroadcastExchange (24) : +- * Project (23) : +- * Filter (22) - : +- * BroadcastHashJoin LeftOuter BuildRight (21) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.item (11) - : +- BroadcastExchange (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Filter (16) - : +- * ColumnarToRow (15) - : +- Scan parquet default.item (14) + : +- * ColumnarToRow (21) + : +- Scan parquet default.date_dim (20) +- * Sort (42) +- Exchange (41) +- * Project (40) @@ -65,112 +65,112 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Condition : ((isnotnull(ss_customer_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_item_sk#2)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : ((isnotnull(d_month_seq#5) AND (d_month_seq#5 = Subquery scalar-subquery#6, [id=#7])) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_customer_sk#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, d_date_sk#4] - -(11) Scan parquet default.item -Output [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(4) Scan parquet default.item +Output [3]: [i_item_sk#4, i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(5) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] -(13) Filter [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] -Condition : (isnotnull(i_current_price#10) AND isnotnull(i_item_sk#9)) +(6) Filter [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] +Condition : (isnotnull(i_current_price#5) AND isnotnull(i_item_sk#4)) -(14) Scan parquet default.item -Output [2]: [i_current_price#10, i_category#11] +(7) Scan parquet default.item +Output [2]: [i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] - -(16) Filter [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Condition : isnotnull(i_category#11) - -(17) HashAggregate [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [i_category#11, sum#14, count#15] - -(18) Exchange -Input [3]: [i_category#11, sum#14, count#15] -Arguments: hashpartitioning(i_category#11, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 3] -Input [3]: [i_category#11, sum#14, count#15] -Keys [1]: [i_category#11] -Functions [1]: [avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#10))#17] -Results [2]: [cast((avg(UnscaledValue(i_current_price#10))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#11 AS i_category#11#19] - -(20) BroadcastExchange -Input [2]: [avg(i_current_price)#18, i_category#11#19] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_category#11] -Right keys [1]: [i_category#11#19] +(8) ColumnarToRow [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] + +(9) Filter [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Condition : isnotnull(i_category#6) + +(10) HashAggregate [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Keys [1]: [i_category#6] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [2]: [sum#7, count#8] +Results [3]: [i_category#6, sum#9, count#10] + +(11) Exchange +Input [3]: [i_category#6, sum#9, count#10] +Arguments: hashpartitioning(i_category#6, 5), true, [id=#11] + +(12) HashAggregate [codegen id : 2] +Input [3]: [i_category#6, sum#9, count#10] +Keys [1]: [i_category#6] +Functions [1]: [avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#5))#12] +Results [2]: [cast((avg(UnscaledValue(i_current_price#5))#12 / 100.0) as decimal(11,6)) AS avg(i_current_price)#13, i_category#6 AS i_category#6#14] + +(13) BroadcastExchange +Input [2]: [avg(i_current_price)#13, i_category#6#14] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#15] + +(14) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#6] +Right keys [1]: [i_category#6#14] +Join condition: None + +(15) Filter [codegen id : 3] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] +Condition : (cast(i_current_price#5 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#13)), DecimalType(14,7), true)) + +(16) Project [codegen id : 3] +Output [1]: [i_item_sk#4] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] + +(17) BroadcastExchange +Input [1]: [i_item_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None +(19) Project [codegen id : 5] +Output [2]: [ss_sold_date_sk#1, ss_customer_sk#3] +Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, i_item_sk#4] + +(20) Scan parquet default.date_dim +Output [2]: [d_date_sk#17, d_month_seq#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(21) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#17, d_month_seq#18] + (22) Filter [codegen id : 4] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] -Condition : (cast(i_current_price#10 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#18)), DecimalType(14,7), true)) +Input [2]: [d_date_sk#17, d_month_seq#18] +Condition : ((isnotnull(d_month_seq#18) AND (d_month_seq#18 = Subquery scalar-subquery#19, [id=#20])) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 4] -Output [1]: [i_item_sk#9] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_month_seq#18] (24) BroadcastExchange -Input [1]: [i_item_sk#9] +Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#3] -Input [3]: [ss_item_sk#2, ss_customer_sk#3, i_item_sk#9] +Input [3]: [ss_sold_date_sk#1, ss_customer_sk#3, d_date_sk#17] (27) Exchange Input [1]: [ss_customer_sk#3] @@ -282,7 +282,7 @@ Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#6, [id=#7] +Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#19, [id=#20] * HashAggregate (57) +- Exchange (56) +- * HashAggregate (55) @@ -293,39 +293,39 @@ Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery (51) Scan parquet default.date_dim -Output [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [3]: [d_month_seq#18, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (52) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (53) Filter [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] Condition : (((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2000)) AND (d_moy#38 = 1)) (54) Project [codegen id : 1] -Output [1]: [d_month_seq#5] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [1]: [d_month_seq#18] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (55) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] (56) Exchange -Input [1]: [d_month_seq#5] -Arguments: hashpartitioning(d_month_seq#5, 5), true, [id=#39] +Input [1]: [d_month_seq#18] +Arguments: hashpartitioning(d_month_seq#18, 5), true, [id=#39] (57) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index dcebba331afb..73d42163240f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -16,55 +16,55 @@ TakeOrderedAndProject [cnt,state] Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Project [ss_customer_sk] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_customer_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [d_month_seq] + WholeStageCodegen (3) + Project [i_item_sk] + Filter [i_current_price,avg(i_current_price)] + BroadcastHashJoin [i_category,i_category] + Filter [i_current_price,i_item_sk] + ColumnarToRow InputAdapter - Exchange [d_month_seq] #4 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] + BroadcastExchange #4 + WholeStageCodegen (2) + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + InputAdapter + Exchange [i_category] #5 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_current_price,i_category] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - Project [i_item_sk] - Filter [i_current_price,avg(i_current_price)] - BroadcastHashJoin [i_category,i_category] - Filter [i_current_price,i_item_sk] - ColumnarToRow + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [d_month_seq] InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_category] + Exchange [d_month_seq] #7 + WholeStageCodegen (1) + HashAggregate [d_month_seq] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] - InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_current_price,i_category] + Scan parquet default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index e616934bbd07..58a60763b2b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- BroadcastNestedLoopJoin Inner BuildRight (73) +TakeOrderedAndProject (69) ++- * Project (68) + +- BroadcastNestedLoopJoin Inner BuildRight (67) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -49,31 +49,25 @@ TakeOrderedAndProject (75) : +- * Filter (37) : +- * ColumnarToRow (36) : +- Scan parquet default.customer_address (35) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) + +- BroadcastExchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) :- * Project (59) : +- * BroadcastHashJoin Inner BuildRight (58) : :- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildLeft (52) - : : : :- ReusedExchange (48) - : : : +- * Filter (51) - : : : +- * ColumnarToRow (50) - : : : +- Scan parquet default.store_sales (49) + : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : :- * Filter (50) + : : : : +- * ColumnarToRow (49) + : : : : +- Scan parquet default.store_sales (48) + : : : +- ReusedExchange (51) : : +- ReusedExchange (54) : +- ReusedExchange (57) - +- BroadcastExchange (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildLeft (64) - :- ReusedExchange (60) - +- * Filter (63) - +- * ColumnarToRow (62) - +- Scan parquet default.customer (61) + +- ReusedExchange (60) (1) Scan parquet default.store_sales @@ -290,31 +284,31 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#31] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#31,17,2) AS promotions#32] -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#7] - -(49) Scan parquet default.store_sales +(48) 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#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(50) ColumnarToRow +(49) ColumnarToRow [codegen id : 14] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] -(51) Filter +(50) Filter [codegen id : 14] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_item_sk#2)) +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#7] + (52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [d_date_sk#7] -Right keys [1]: [ss_sold_date_sk#1] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#7] Join condition: None (53) Project [codegen id : 14] Output [4]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] -Input [6]: [d_date_sk#7, ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6] +Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6, d_date_sk#7] (54) ReusedExchange [Reuses operator id: 15] Output [1]: [i_item_sk#11] @@ -340,75 +334,48 @@ Join condition: None Output [2]: [ss_customer_sk#3, ss_ext_sales_price#6] Input [4]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#6, s_store_sk#19] -(60) ReusedExchange [Reuses operator id: 39] -Output [1]: [ca_address_sk#24] - -(61) Scan parquet default.customer -Output [2]: [c_customer_sk#22, c_current_addr_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(62) ColumnarToRow -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] - -(63) Filter -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#23] -Join condition: None - -(65) Project [codegen id : 13] +(60) ReusedExchange [Reuses operator id: 42] Output [1]: [c_customer_sk#22] -Input [3]: [ca_address_sk#24, c_customer_sk#22, c_current_addr_sk#23] - -(66) BroadcastExchange -Input [1]: [c_customer_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] -(67) BroadcastHashJoin [codegen id : 14] +(61) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_customer_sk#3] Right keys [1]: [c_customer_sk#22] Join condition: None -(68) Project [codegen id : 14] +(62) Project [codegen id : 14] Output [1]: [ss_ext_sales_price#6] Input [3]: [ss_customer_sk#3, ss_ext_sales_price#6, c_customer_sk#22] -(69) HashAggregate [codegen id : 14] +(63) HashAggregate [codegen id : 14] Input [1]: [ss_ext_sales_price#6] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Aggregate Attributes [1]: [sum#33] +Results [1]: [sum#34] -(70) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, true, [id=#36] +(64) Exchange +Input [1]: [sum#34] +Arguments: SinglePartition, true, [id=#35] -(71) HashAggregate [codegen id : 15] -Input [1]: [sum#35] +(65) HashAggregate [codegen id : 15] +Input [1]: [sum#34] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#37,17,2) AS total#38] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#6))#36] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#36,17,2) AS total#37] -(72) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +(66) BroadcastExchange +Input [1]: [total#37] +Arguments: IdentityBroadcastMode, [id=#38] -(73) BroadcastNestedLoopJoin +(67) BroadcastNestedLoopJoin Join condition: None -(74) Project [codegen id : 16] -Output [3]: [promotions#32, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#32 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#40] -Input [2]: [promotions#32, total#38] +(68) Project [codegen id : 16] +Output [3]: [promotions#32, total#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#32 as decimal(15,4))) / promote_precision(cast(total#37 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] +Input [2]: [promotions#32, total#37] -(75) TakeOrderedAndProject -Input [3]: [promotions#32, total#38, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#40] -Arguments: 100, [promotions#32 ASC NULLS FIRST, total#38 ASC NULLS FIRST], [promotions#32, total#38, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#40] +(69) TakeOrderedAndProject +Input [3]: [promotions#32, total#37, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] +Arguments: 100, [promotions#32 ASC NULLS FIRST, total#37 ASC NULLS FIRST], [promotions#32, total#37, (CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt index 039ccb1aa18c..87f2b3ae0374 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt @@ -86,25 +86,16 @@ TakeOrderedAndProject [promotions,total,(CAST((CAST(CAST(promotions AS DECIMAL(1 Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_store_sk,ss_sold_date_sk,ss_customer_sk,ss_item_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 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk] #3 InputAdapter ReusedExchange [s_store_sk] #5 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (13) - Project [c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - InputAdapter - ReusedExchange [ca_address_sk] #7 - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + ReusedExchange [c_customer_sk] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt index e9a2b7a375b0..b74dfb49c9f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject (32) : :- * 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.web_sales (6) + : : : +- * 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) : : +- BroadcastExchange (14) : : +- * Filter (13) : : +- * ColumnarToRow (12) @@ -33,50 +33,50 @@ TakeOrderedAndProject (32) +- Scan parquet default.warehouse (23) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_month_seq#2] +(1) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(2) ColumnarToRow [codegen id : 5] +Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] -Condition : (((isnotnull(d_month_seq#2) AND (d_month_seq#2 >= 1200)) AND (d_month_seq#2 <= 1211)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] +Condition : (((isnotnull(ws_warehouse_sk#5) AND isnotnull(ws_ship_mode_sk#4)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] -(6) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(7) ColumnarToRow -Input [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#6] +Input [2]: [d_date_sk#6, d_month_seq#7] -(8) Filter -Input [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] -Condition : (((isnotnull(ws_warehouse_sk#8) AND isnotnull(ws_ship_mode_sk#7)) AND isnotnull(ws_web_site_sk#6)) AND isnotnull(ws_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ws_ship_date_sk#5] +Left keys [1]: [ws_ship_date_sk#2] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] -Input [6]: [d_date_sk#1, ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5] +Input [6]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5, d_date_sk#6] (11) Scan parquet default.web_site Output [2]: [web_site_sk#9, web_name#10] @@ -97,13 +97,13 @@ Input [2]: [web_site_sk#9, web_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_web_site_sk#6] +Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#9] Join condition: None (16) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_ship_mode_sk#7, ws_warehouse_sk#8, web_name#10] -Input [7]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_web_site_sk#6, ws_ship_mode_sk#7, ws_warehouse_sk#8, web_site_sk#9, web_name#10] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_ship_mode_sk#4, ws_warehouse_sk#5, web_name#10] +Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_web_site_sk#3, ws_ship_mode_sk#4, ws_warehouse_sk#5, web_site_sk#9, web_name#10] (17) Scan parquet default.ship_mode Output [2]: [sm_ship_mode_sk#12, sm_type#13] @@ -124,13 +124,13 @@ Input [2]: [sm_ship_mode_sk#12, sm_type#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#7] +Left keys [1]: [ws_ship_mode_sk#4] Right keys [1]: [sm_ship_mode_sk#12] Join condition: None (22) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_warehouse_sk#8, web_name#10, sm_type#13] -Input [7]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_ship_mode_sk#7, ws_warehouse_sk#8, web_name#10, sm_ship_mode_sk#12, sm_type#13] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_warehouse_sk#5, web_name#10, sm_type#13] +Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_ship_mode_sk#4, ws_warehouse_sk#5, web_name#10, sm_ship_mode_sk#12, sm_type#13] (23) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#15, w_warehouse_name#16] @@ -151,18 +151,18 @@ Input [2]: [w_warehouse_sk#15, w_warehouse_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#8] +Left keys [1]: [ws_warehouse_sk#5] Right keys [1]: [w_warehouse_sk#15] Join condition: None (28) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, w_warehouse_name#16, sm_type#13, web_name#10] -Input [7]: [ws_sold_date_sk#4, ws_ship_date_sk#5, ws_warehouse_sk#8, web_name#10, sm_type#13, w_warehouse_sk#15, w_warehouse_name#16] +Output [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#16, sm_type#13, web_name#10] +Input [7]: [ws_sold_date_sk#1, ws_ship_date_sk#2, ws_warehouse_sk#5, web_name#10, sm_type#13, w_warehouse_sk#15, w_warehouse_name#16] (29) HashAggregate [codegen id : 5] -Input [5]: [ws_sold_date_sk#4, ws_ship_date_sk#5, w_warehouse_name#16, sm_type#13, web_name#10] +Input [5]: [ws_sold_date_sk#1, ws_ship_date_sk#2, w_warehouse_name#16, sm_type#13, web_name#10] Keys [3]: [substr(w_warehouse_name#16, 1, 20) AS substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10] -Functions [5]: [partial_sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] +Functions [5]: [partial_sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] Aggregate Attributes [5]: [sum#19, sum#20, sum#21, sum#22, sum#23] Results [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10, sum#24, sum#25, sum#26, sum#27, sum#28] @@ -173,9 +173,9 @@ Arguments: hashpartitioning(substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, w (31) HashAggregate [codegen id : 6] Input [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10, sum#24, sum#25, sum#26, sum#27, sum#28] Keys [3]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#13, web_name#10] -Functions [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34] -Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 30) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 60) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 90) AND ((ws_ship_date_sk#5 - ws_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((ws_ship_date_sk#5 - ws_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] +Functions [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] +Aggregate Attributes [5]: [sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34] +Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 30) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 60) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 90) AND ((ws_ship_date_sk#2 - ws_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((ws_ship_date_sk#2 - ws_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt index 59cfc4b7b249..9b16b44792ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt @@ -12,7 +12,11 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days Project [ws_sold_date_sk,ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,web_name] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] - Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 4b863587b08d..5db04537d637 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject (55) : : :- * Project (17) : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : :- BroadcastExchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.ship_mode (1) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.web_sales (6) + : : : : +- * 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.ship_mode (4) : : : +- BroadcastExchange (15) : : : +- * Project (14) : : : +- * Filter (13) @@ -46,60 +46,60 @@ TakeOrderedAndProject (55) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildLeft (37) - : : : :- ReusedExchange (33) - : : : +- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.catalog_sales (34) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.catalog_sales (33) + : : : +- ReusedExchange (36) : : +- ReusedExchange (39) : +- ReusedExchange (42) +- ReusedExchange (45) -(1) Scan parquet default.ship_mode -Output [2]: [sm_ship_mode_sk#1, sm_carrier#2] +(1) Scan parquet default.web_sales +Output [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [In(sm_carrier, [DHL,BARIAN]), IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [sm_ship_mode_sk#1, sm_carrier#2] +(2) ColumnarToRow [codegen id : 5] +Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] -(3) Filter [codegen id : 1] -Input [2]: [sm_ship_mode_sk#1, sm_carrier#2] -Condition : (sm_carrier#2 IN (DHL,BARIAN) AND isnotnull(sm_ship_mode_sk#1)) +(3) Filter [codegen id : 5] +Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] +Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_sold_date_sk#1)) AND isnotnull(ws_sold_time_sk#2)) AND isnotnull(ws_ship_mode_sk#3)) -(4) Project [codegen id : 1] -Output [1]: [sm_ship_mode_sk#1] -Input [2]: [sm_ship_mode_sk#1, sm_carrier#2] +(4) Scan parquet default.ship_mode +Output [2]: [sm_ship_mode_sk#8, sm_carrier#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/ship_mode] +PushedFilters: [In(sm_carrier, [DHL,BARIAN]), IsNotNull(sm_ship_mode_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [sm_ship_mode_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [sm_ship_mode_sk#8, sm_carrier#9] -(6) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [sm_ship_mode_sk#8, sm_carrier#9] +Condition : (sm_carrier#9 IN (DHL,BARIAN) AND isnotnull(sm_ship_mode_sk#8)) -(7) ColumnarToRow -Input [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] +(7) Project [codegen id : 1] +Output [1]: [sm_ship_mode_sk#8] +Input [2]: [sm_ship_mode_sk#8, sm_carrier#9] -(8) Filter -Input [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Condition : (((isnotnull(ws_warehouse_sk#7) AND isnotnull(ws_sold_date_sk#4)) AND isnotnull(ws_sold_time_sk#5)) AND isnotnull(ws_ship_mode_sk#6)) +(8) BroadcastExchange +Input [1]: [sm_ship_mode_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sm_ship_mode_sk#1] -Right keys [1]: [ws_ship_mode_sk#6] +Left keys [1]: [ws_ship_mode_sk#3] +Right keys [1]: [sm_ship_mode_sk#8] Join condition: None (10) Project [codegen id : 5] -Output [6]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Input [8]: [sm_ship_mode_sk#1, ws_sold_date_sk#4, ws_sold_time_sk#5, ws_ship_mode_sk#6, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] +Output [6]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] +Input [8]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, sm_ship_mode_sk#8] (11) Scan parquet default.time_dim Output [2]: [t_time_sk#11, t_time#12] @@ -124,13 +124,13 @@ Input [1]: [t_time_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#5] +Left keys [1]: [ws_sold_time_sk#2] Right keys [1]: [t_time_sk#11] Join condition: None (17) Project [codegen id : 5] -Output [5]: [ws_sold_date_sk#4, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10] -Input [7]: [ws_sold_date_sk#4, ws_sold_time_sk#5, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, t_time_sk#11] +Output [5]: [ws_sold_date_sk#1, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7] +Input [7]: [ws_sold_date_sk#1, ws_sold_time_sk#2, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, t_time_sk#11] (18) Scan parquet default.date_dim Output [3]: [d_date_sk#14, d_year#15, d_moy#16] @@ -151,13 +151,13 @@ Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#4] +Left keys [1]: [ws_sold_date_sk#1] Right keys [1]: [d_date_sk#14] Join condition: None (23) Project [codegen id : 5] -Output [6]: [ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, d_year#15, d_moy#16] -Input [8]: [ws_sold_date_sk#4, ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, d_date_sk#14, d_year#15, d_moy#16] +Output [6]: [ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, d_year#15, d_moy#16] +Input [8]: [ws_sold_date_sk#1, ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, d_date_sk#14, d_year#15, d_moy#16] (24) Scan parquet default.warehouse Output [7]: [w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24] @@ -178,18 +178,18 @@ Input [7]: [w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#7] +Left keys [1]: [ws_warehouse_sk#4] Right keys [1]: [w_warehouse_sk#18] Join condition: None (29) Project [codegen id : 5] -Output [11]: [ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] -Input [13]: [ws_warehouse_sk#7, ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, d_year#15, d_moy#16, w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24] +Output [11]: [ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] +Input [13]: [ws_warehouse_sk#4, ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, d_year#15, d_moy#16, w_warehouse_sk#18, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24] (30) HashAggregate [codegen id : 5] -Input [11]: [ws_quantity#8, ws_ext_sales_price#9, ws_net_paid#10, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] +Input [11]: [ws_quantity#5, ws_ext_sales_price#6, ws_net_paid#7, w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, d_moy#16] Keys [7]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15] -Functions [24]: [partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Functions [24]: [partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] Aggregate Attributes [48]: [sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71, sum#72, isEmpty#73] Results [55]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] @@ -200,35 +200,35 @@ Arguments: hashpartitioning(w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21 (32) HashAggregate [codegen id : 6] Input [55]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] Keys [7]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, d_year#15] -Functions [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146] -Results [32]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, DHL,BARIAN AS ship_carriers#147, d_year#15 AS year#148, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123 AS jan_sales#149, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124 AS feb_sales#150, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125 AS mar_sales#151, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126 AS apr_sales#152, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127 AS may_sales#153, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128 AS jun_sales#154, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129 AS jul_sales#155, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130 AS aug_sales#156, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131 AS sep_sales#157, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132 AS oct_sales#158, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133 AS nov_sales#159, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#9 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134 AS dec_sales#160, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135 AS jan_net#161, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136 AS feb_net#162, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137 AS mar_net#163, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138 AS apr_net#164, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139 AS may_net#165, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140 AS jun_net#166, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141 AS jul_net#167, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142 AS aug_net#168, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143 AS sep_net#169, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144 AS oct_net#170, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145 AS nov_net#171, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#10 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#8 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146 AS dec_net#172] +Functions [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146] +Results [32]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, DHL,BARIAN AS ship_carriers#147, d_year#15 AS year#148, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#123 AS jan_sales#149, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#124 AS feb_sales#150, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#125 AS mar_sales#151, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#126 AS apr_sales#152, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#127 AS may_sales#153, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#128 AS jun_sales#154, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#129 AS jul_sales#155, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#130 AS aug_sales#156, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#131 AS sep_sales#157, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#132 AS oct_sales#158, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#133 AS nov_sales#159, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#6 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#134 AS dec_sales#160, sum(CASE WHEN (d_moy#16 = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#135 AS jan_net#161, sum(CASE WHEN (d_moy#16 = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#136 AS feb_net#162, sum(CASE WHEN (d_moy#16 = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#137 AS mar_net#163, sum(CASE WHEN (d_moy#16 = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#138 AS apr_net#164, sum(CASE WHEN (d_moy#16 = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#139 AS may_net#165, sum(CASE WHEN (d_moy#16 = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#140 AS jun_net#166, sum(CASE WHEN (d_moy#16 = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#141 AS jul_net#167, sum(CASE WHEN (d_moy#16 = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#142 AS aug_net#168, sum(CASE WHEN (d_moy#16 = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#143 AS sep_net#169, sum(CASE WHEN (d_moy#16 = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#144 AS oct_net#170, sum(CASE WHEN (d_moy#16 = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#145 AS nov_net#171, sum(CASE WHEN (d_moy#16 = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid#7 as decimal(12,2))) * promote_precision(cast(cast(ws_quantity#5 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#146 AS dec_net#172] -(33) ReusedExchange [Reuses operator id: 5] -Output [1]: [sm_ship_mode_sk#1] - -(34) Scan parquet default.catalog_sales +(33) Scan parquet default.catalog_sales Output [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(35) ColumnarToRow +(34) ColumnarToRow [codegen id : 11] Input [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] -(36) Filter +(35) Filter [codegen id : 11] Input [7]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] Condition : (((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_date_sk#173)) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) +(36) ReusedExchange [Reuses operator id: 8] +Output [1]: [sm_ship_mode_sk#8] + (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [sm_ship_mode_sk#1] -Right keys [1]: [cs_ship_mode_sk#175] +Left keys [1]: [cs_ship_mode_sk#175] +Right keys [1]: [sm_ship_mode_sk#8] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] -Input [8]: [sm_ship_mode_sk#1, cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179] +Input [8]: [cs_sold_date_sk#173, cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, sm_ship_mode_sk#8] (39) ReusedExchange [Reuses operator id: 15] Output [1]: [t_time_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt index 465d269a847c..ddfb04d8df5e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt @@ -20,7 +20,11 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Project [ws_sold_date_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] BroadcastHashJoin [ws_sold_time_sk,t_time_sk] Project [ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] - BroadcastHashJoin [sm_ship_mode_sk,ws_ship_mode_sk] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -29,10 +33,6 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter Scan parquet default.ship_mode [sm_ship_mode_sk,sm_carrier] - Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -68,13 +68,13 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Project [cs_sold_date_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] BroadcastHashJoin [cs_sold_time_sk,t_time_sk] Project [cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] - BroadcastHashJoin [sm_ship_mode_sk,cs_ship_mode_sk] - InputAdapter - ReusedExchange [sm_ship_mode_sk] #3 + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] Filter [cs_warehouse_sk,cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] + InputAdapter + ReusedExchange [sm_ship_mode_sk] #3 InputAdapter ReusedExchange [t_time_sk] #4 InputAdapter 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 a100b6659f16..3f8106c96379 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 @@ -24,15 +24,15 @@ TakeOrderedAndProject (79) : : : : : :- * Project (17) : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : :- BroadcastExchange (5) - : : : : : : : : +- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.household_demographics (1) - : : : : : : : +- * Filter (8) - : : : : : : : +- * ColumnarToRow (7) - : : : : : : : +- Scan parquet default.catalog_sales (6) + : : : : : : : +- * 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) @@ -49,26 +49,26 @@ TakeOrderedAndProject (79) : : : : +- Scan parquet default.item (26) : : : +- BroadcastExchange (43) : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- BroadcastExchange (40) - : : : +- * Project (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.date_dim (36) + : : : +- * 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 BuildLeft (55) - : : :- BroadcastExchange (51) - : : : +- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.warehouse (48) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet default.inventory (52) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) + : : +- BroadcastExchange (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet default.warehouse (51) : +- BroadcastExchange (64) : +- * Filter (63) : +- * ColumnarToRow (62) @@ -80,50 +80,50 @@ TakeOrderedAndProject (79) +- Scan parquet default.catalog_returns (69) -(1) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#1, hd_buy_potential#2] +(1) Scan parquet default.catalog_sales +Output [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(2) ColumnarToRow [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] -(3) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] -Condition : ((isnotnull(hd_buy_potential#2) AND (hd_buy_potential#2 = >10000)) AND isnotnull(hd_demo_sk#1)) +(3) Filter [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnotnull(cs_bill_cdemo_sk#3)) AND isnotnull(cs_bill_hdemo_sk#4)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [hd_demo_sk#1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(4) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [hd_demo_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(6) Scan parquet default.catalog_sales -Output [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct +(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)) -(7) ColumnarToRow -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +(7) Project [codegen id : 1] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(8) Filter -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Condition : (((((isnotnull(cs_quantity#11) AND isnotnull(cs_item_sk#8)) AND isnotnull(cs_bill_cdemo_sk#6)) AND isnotnull(cs_bill_hdemo_sk#7)) AND isnotnull(cs_sold_date_sk#4)) AND isnotnull(cs_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [hd_demo_sk#1] -Right keys [1]: [cs_bill_hdemo_sk#7] +Left keys [1]: [cs_bill_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#9] Join condition: None (10) Project [codegen id : 4] -Output [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [9]: [hd_demo_sk#1, cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +Output [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, hd_demo_sk#9] (11) Scan parquet default.customer_demographics Output [2]: [cd_demo_sk#12, cd_marital_status#13] @@ -148,13 +148,13 @@ Input [1]: [cd_demo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#6] +Left keys [1]: [cs_bill_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, cd_demo_sk#12] +Output [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, cd_demo_sk#12] (18) Scan parquet default.date_dim Output [2]: [d_date_sk#15, d_date#16] @@ -175,21 +175,21 @@ Input [2]: [d_date_sk#15, d_date#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_ship_date_sk#5] +Left keys [1]: [cs_ship_date_sk#2] Right keys [1]: [d_date_sk#15] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date_sk#15, d_date#16] +Output [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date_sk#15, d_date#16] (24) Exchange -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: hashpartitioning(cs_item_sk#8, 5), true, [id=#18] +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] (25) Sort [codegen id : 5] -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: [cs_item_sk#8 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: [cs_item_sk#5 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item Output [2]: [i_item_sk#19, i_item_desc#20] @@ -214,137 +214,137 @@ Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 (31) SortMergeJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#8] +Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None (32) Project [codegen id : 10] -Output [7]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20] -Input [8]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_sk#19, i_item_desc#20] +Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_week_seq#23] +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_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct -(34) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(35) Filter [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] -Condition : (isnotnull(d_week_seq#23) AND isnotnull(d_date_sk#22)) +(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)) -(36) Scan parquet default.date_dim -Output [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -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) 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] -(37) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#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] -(38) Filter [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -Condition : ((((isnotnull(d_year#27) AND (d_year#27 = 1999)) AND isnotnull(d_date_sk#24)) AND isnotnull(d_week_seq#26)) AND isnotnull(d_date#25)) +(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 -(39) Project [codegen id : 8] -Output [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(40) BroadcastExchange -Input [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#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)) (41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#23] -Right keys [1]: [d_week_seq#26] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None (42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] -Input [5]: [d_date_sk#22, d_week_seq#23, d_date_sk#24, d_date#25, d_week_seq#26] +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_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#29] +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#4] -Right keys [1]: [d_date_sk#24] -Join condition: (d_date#16 > d_date#25 + 5 days) +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#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Input [11]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20, d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] +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] (46) Exchange -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, d_date_sk#22, 5), true, [id=#30] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] (47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, d_date_sk#22 ASC NULLS FIRST], false, 0 +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.warehouse -Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] +(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}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +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 -(50) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Condition : isnotnull(w_warehouse_sk#31) +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(51) BroadcastExchange -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(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)) -(52) Scan parquet default.inventory -Output [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(51) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] 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 +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct + +(52) ColumnarToRow [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] -(53) ColumnarToRow -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(53) Filter [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Condition : isnotnull(w_warehouse_sk#35) -(54) Filter -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] -Condition : (((isnotnull(inv_quantity_on_hand#37) AND isnotnull(inv_item_sk#35)) AND isnotnull(inv_warehouse_sk#36)) AND isnotnull(inv_date_sk#34)) +(54) BroadcastExchange +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [w_warehouse_sk#31] -Right keys [1]: [inv_warehouse_sk#36] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Input [6]: [w_warehouse_sk#31, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +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 [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: hashpartitioning(inv_item_sk#35, inv_date_sk#34, 5), true, [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), true, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: [inv_item_sk#35 ASC NULLS FIRST, inv_date_sk#34 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#8, d_date_sk#22] -Right keys [2]: [inv_item_sk#35, inv_date_sk#34] -Join condition: (inv_quantity_on_hand#37 < cs_quantity#11) +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#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [11]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] +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] @@ -365,21 +365,21 @@ Input [1]: [p_promo_sk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (65) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [cs_promo_sk#9] +Left keys [1]: [cs_promo_sk#6] Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, 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#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, cs_order_number#10, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, cs_order_number#10 ASC NULLS FIRST], false, 0 +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 Output [2]: [cr_item_sk#42, cr_order_number#43] @@ -404,33 +404,33 @@ Input [2]: [cr_item_sk#42, cr_order_number#43] Arguments: [cr_item_sk#42 ASC NULLS FIRST, cr_order_number#43 ASC NULLS FIRST], false, 0 (74) SortMergeJoin -Left keys [2]: [cs_item_sk#8, cs_order_number#10] +Left keys [2]: [cs_item_sk#5, cs_order_number#7] Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, 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#32, i_item_desc#20, d_week_seq#26] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +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#32, d_week_seq#26, 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#32, d_week_seq#26, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#32, d_week_seq#26, 5), true, [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), true, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +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#32, d_week_seq#26, 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#32, d_week_seq#26, 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#32 ASC NULLS FIRST, d_week_seq#26 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, 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 39dba3af0235..918508787c4b 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 @@ -23,7 +23,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_date_sk,d_week_seq] + 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] @@ -38,7 +38,11 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom 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 [hd_demo_sk,cs_bill_hdemo_sk] + 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 BroadcastExchange #5 WholeStageCodegen (1) @@ -47,10 +51,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] - 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 BroadcastExchange #6 WholeStageCodegen (2) @@ -79,12 +79,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Project [d_date_sk,d_date_sk,d_date,d_week_seq] + Project [d_date_sk,d_date,d_week_seq,d_date_sk] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -93,14 +89,22 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom 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,inv_date_sk] InputAdapter Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [w_warehouse_name,inv_date_sk,inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [w_warehouse_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) @@ -108,10 +112,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #13 WholeStageCodegen (15) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index 057d786afbcd..9ac081b356c9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (108) : : : +- * Project (23) : : : +- * Filter (22) : : : +- * ColumnarToRow (21) - : : : +- Scan parquet default.date_dim (20) + : : : +- Scan parquet default.promotion (20) : : +- BroadcastExchange (31) : : +- * Project (30) : : +- * Filter (29) : : +- * ColumnarToRow (28) - : : +- Scan parquet default.promotion (27) + : : +- Scan parquet default.date_dim (27) : +- BroadcastExchange (37) : +- * Filter (36) : +- * ColumnarToRow (35) @@ -193,67 +193,67 @@ Join condition: None Output [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, i_item_sk#14] -(20) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +(20) Scan parquet default.promotion +Output [2]: [p_promo_sk#17, p_channel_tv#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] +ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (22) Filter [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 11192)) AND (d_date#18 <= 11222)) AND isnotnull(d_date_sk#17)) +Input [2]: [p_promo_sk#17, p_channel_tv#18] +Condition : ((isnotnull(p_channel_tv#18) AND (p_channel_tv#18 = N)) AND isnotnull(p_promo_sk#17)) (23) Project [codegen id : 6] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [p_promo_sk#17] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (24) BroadcastExchange -Input [1]: [d_date_sk#17] +Input [1]: [p_promo_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_promo_sk#4] +Right keys [1]: [p_promo_sk#17] Join condition: None (26) Project [codegen id : 9] -Output [6]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#17] +Output [6]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] +Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#17] -(27) Scan parquet default.promotion -Output [2]: [p_promo_sk#20, p_channel_tv#21] +(27) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_date#21] Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] +ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Input [2]: [d_date_sk#20, d_date#21] (29) Filter [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) +Input [2]: [d_date_sk#20, d_date#21] +Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 11192)) AND (d_date#21 <= 11222)) AND isnotnull(d_date_sk#20)) (30) Project [codegen id : 7] -Output [1]: [p_promo_sk#20] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_date#21] (31) BroadcastExchange -Input [1]: [p_promo_sk#20] +Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#20] Join condition: None (33) Project [codegen id : 9] Output [5]: [ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [7]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#20] +Input [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#20] (34) Scan parquet default.store Output [2]: [s_store_sk#23, s_store_id#24] @@ -366,28 +366,28 @@ Output [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_s Input [9]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_item_sk#47, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, i_item_sk#14] (58) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (59) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_promo_sk#48] +Right keys [1]: [p_promo_sk#17] Join condition: None (60) Project [codegen id : 19] -Output [6]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#17] +Output [6]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] +Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#17] (61) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (62) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#48] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [cs_sold_date_sk#45] +Right keys [1]: [d_date_sk#20] Join condition: None (63) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [7]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#20] +Input [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#20] (64) Scan parquet default.catalog_page Output [2]: [cp_catalog_page_sk#58, cp_catalog_page_id#59] @@ -500,28 +500,28 @@ Output [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales Input [9]: [ws_sold_date_sk#80, ws_item_sk#81, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, i_item_sk#14] (88) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_promo_sk#83] +Right keys [1]: [p_promo_sk#17] Join condition: None (90) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#17] +Output [6]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] +Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#17] (91) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#83] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#20] Join condition: None (93) Project [codegen id : 29] Output [5]: [ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [7]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#20] +Input [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#20] (94) Scan parquet default.web_site Output [2]: [web_site_sk#93, web_site_id#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt index 7b73e4307dcf..ec00b49e7198 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt @@ -17,9 +17,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] Project [ss_sold_date_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] @@ -54,19 +54,19 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) - Project [d_date_sk] - Filter [d_date,d_date_sk] + Project [p_promo_sk] + Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Project [p_promo_sk] - Filter [p_channel_tv,p_promo_sk] + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_tv] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) @@ -83,9 +83,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] @@ -112,9 +112,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [p_promo_sk] #6 InputAdapter - ReusedExchange [p_promo_sk] #7 + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #12 WholeStageCodegen (18) @@ -131,9 +131,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] Project [ws_sold_date_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] @@ -160,9 +160,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [p_promo_sk] #6 InputAdapter - ReusedExchange [p_promo_sk] #7 + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #16 WholeStageCodegen (28) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt index ae0b996ec28b..83ec6391d773 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt @@ -20,15 +20,15 @@ TakeOrderedAndProject (36) : : : +- Scan parquet default.customer_address (4) : : +- BroadcastExchange (21) : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildLeft (19) - : : :- BroadcastExchange (15) - : : : +- * Project (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.income_band (11) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.household_demographics (16) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Filter (13) + : : : +- * ColumnarToRow (12) + : : : +- Scan parquet default.household_demographics (11) + : : +- BroadcastExchange (18) + : : +- * Project (17) + : : +- * Filter (16) + : : +- * ColumnarToRow (15) + : : +- Scan parquet default.income_band (14) : +- * Filter (27) : +- * ColumnarToRow (26) : +- Scan parquet default.customer_demographics (25) @@ -82,63 +82,63 @@ Join condition: None Output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] Input [7]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6, ca_address_sk#7] -(11) Scan parquet default.income_band -Output [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] +(11) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#10, hd_income_band_sk#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] +ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] +(12) ColumnarToRow [codegen id : 3] +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] -(13) Filter [codegen id : 2] -Input [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] -Condition : ((((isnotnull(ib_lower_bound#11) AND isnotnull(ib_upper_bound#12)) AND (ib_lower_bound#11 >= 38128)) AND (ib_upper_bound#12 <= 88128)) AND isnotnull(ib_income_band_sk#10)) +(13) Filter [codegen id : 3] +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Condition : (isnotnull(hd_demo_sk#10) AND isnotnull(hd_income_band_sk#11)) -(14) Project [codegen id : 2] -Output [1]: [ib_income_band_sk#10] -Input [3]: [ib_income_band_sk#10, ib_lower_bound#11, ib_upper_bound#12] +(14) Scan parquet default.income_band +Output [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/income_band] +PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] +ReadSchema: struct -(15) BroadcastExchange -Input [1]: [ib_income_band_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(15) ColumnarToRow [codegen id : 2] +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] -(16) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#14, hd_income_band_sk#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct +(16) Filter [codegen id : 2] +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Condition : ((((isnotnull(ib_lower_bound#13) AND isnotnull(ib_upper_bound#14)) AND (ib_lower_bound#13 >= 38128)) AND (ib_upper_bound#14 <= 88128)) AND isnotnull(ib_income_band_sk#12)) -(17) ColumnarToRow -Input [2]: [hd_demo_sk#14, hd_income_band_sk#15] +(17) Project [codegen id : 2] +Output [1]: [ib_income_band_sk#12] +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] -(18) Filter -Input [2]: [hd_demo_sk#14, hd_income_band_sk#15] -Condition : (isnotnull(hd_demo_sk#14) AND isnotnull(hd_income_band_sk#15)) +(18) BroadcastExchange +Input [1]: [ib_income_band_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ib_income_band_sk#10] -Right keys [1]: [hd_income_band_sk#15] +Left keys [1]: [hd_income_band_sk#11] +Right keys [1]: [ib_income_band_sk#12] Join condition: None (20) Project [codegen id : 3] -Output [1]: [hd_demo_sk#14] -Input [3]: [ib_income_band_sk#10, hd_demo_sk#14, hd_income_band_sk#15] +Output [1]: [hd_demo_sk#10] +Input [3]: [hd_demo_sk#10, hd_income_band_sk#11, ib_income_band_sk#12] (21) BroadcastExchange -Input [1]: [hd_demo_sk#14] +Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#14] +Right keys [1]: [hd_demo_sk#10] Join condition: None (23) Project [codegen id : 4] Output [4]: [c_customer_id#1, c_current_cdemo_sk#2, c_first_name#5, c_last_name#6] -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, hd_demo_sk#14] +Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, hd_demo_sk#10] (24) BroadcastExchange Input [4]: [c_customer_id#1, c_current_cdemo_sk#2, c_first_name#5, c_last_name#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt index 1fbc57ee7e47..16087526bc13 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt @@ -30,7 +30,11 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - BroadcastHashJoin [ib_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Filter [hd_demo_sk,hd_income_band_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -39,10 +43,6 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] ColumnarToRow InputAdapter Scan parquet default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - Filter [hd_demo_sk,hd_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] Filter [cd_demo_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index ee550f1af494..7c3f00d33f24 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -12,30 +12,30 @@ TakeOrderedAndProject (57) : :- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildLeft (24) - : : : :- BroadcastExchange (5) - : : : : +- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.date_dim (1) - : : : +- * Project (23) - : : : +- * SortMergeJoin Inner (22) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.web_sales (6) - : : : : +- BroadcastExchange (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.web_page (9) - : : : +- * Sort (21) - : : : +- Exchange (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.web_returns (17) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * Project (18) + : : : : +- * SortMergeJoin Inner (17) + : : : : :- * Sort (11) + : : : : : +- Exchange (10) + : : : : : +- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.web_sales (1) + : : : : : +- BroadcastExchange (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.web_page (4) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet default.web_returns (12) + : : : +- BroadcastExchange (23) + : : : +- * Project (22) + : : : +- * Filter (21) + : : : +- * ColumnarToRow (20) + : : : +- Scan parquet default.date_dim (19) : : +- BroadcastExchange (29) : : +- * Filter (28) : : +- * ColumnarToRow (27) @@ -48,126 +48,126 @@ TakeOrderedAndProject (57) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (43) - : +- * ColumnarToRow (42) - : +- Scan parquet default.customer_demographics (41) - +- BroadcastExchange (47) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.customer_demographics (44) - - -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_year#2] -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 - -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] - -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_year#2] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_year#2] - -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] - -(6) Scan parquet default.web_sales -Output [7]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] + +- * BroadcastHashJoin Inner BuildLeft (48) + :- BroadcastExchange (44) + : +- * Filter (43) + : +- * ColumnarToRow (42) + : +- Scan parquet default.customer_demographics (41) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.customer_demographics (45) + + +(1) Scan parquet default.web_sales +Output [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), IsNotNull(ws_sold_date_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] -Input [7]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] +(2) ColumnarToRow [codegen id : 2] +Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] -(8) Filter [codegen id : 3] -Input [7]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Condition : (((((isnotnull(ws_item_sk#5) AND isnotnull(ws_order_number#7)) AND isnotnull(ws_web_page_sk#6)) AND isnotnull(ws_sold_date_sk#4)) AND ((((ws_sales_price#9 >= 100.00) AND (ws_sales_price#9 <= 150.00)) OR ((ws_sales_price#9 >= 50.00) AND (ws_sales_price#9 <= 100.00))) OR ((ws_sales_price#9 >= 150.00) AND (ws_sales_price#9 <= 200.00)))) AND ((((ws_net_profit#10 >= 100.00) AND (ws_net_profit#10 <= 200.00)) OR ((ws_net_profit#10 >= 150.00) AND (ws_net_profit#10 <= 300.00))) OR ((ws_net_profit#10 >= 50.00) AND (ws_net_profit#10 <= 250.00)))) +(3) Filter [codegen id : 2] +Input [7]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Condition : (((((isnotnull(ws_item_sk#2) AND isnotnull(ws_order_number#4)) AND isnotnull(ws_web_page_sk#3)) AND isnotnull(ws_sold_date_sk#1)) AND ((((ws_sales_price#6 >= 100.00) AND (ws_sales_price#6 <= 150.00)) OR ((ws_sales_price#6 >= 50.00) AND (ws_sales_price#6 <= 100.00))) OR ((ws_sales_price#6 >= 150.00) AND (ws_sales_price#6 <= 200.00)))) AND ((((ws_net_profit#7 >= 100.00) AND (ws_net_profit#7 <= 200.00)) OR ((ws_net_profit#7 >= 150.00) AND (ws_net_profit#7 <= 300.00))) OR ((ws_net_profit#7 >= 50.00) AND (ws_net_profit#7 <= 250.00)))) -(9) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#11] +(4) Scan parquet default.web_page +Output [1]: [wp_web_page_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 2] -Input [1]: [wp_web_page_sk#11] +(5) ColumnarToRow [codegen id : 1] +Input [1]: [wp_web_page_sk#8] -(11) Filter [codegen id : 2] -Input [1]: [wp_web_page_sk#11] -Condition : isnotnull(wp_web_page_sk#11) +(6) Filter [codegen id : 1] +Input [1]: [wp_web_page_sk#8] +Condition : isnotnull(wp_web_page_sk#8) -(12) BroadcastExchange -Input [1]: [wp_web_page_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(7) BroadcastExchange +Input [1]: [wp_web_page_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#9] -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_web_page_sk#6] -Right keys [1]: [wp_web_page_sk#11] +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#8] Join condition: None -(14) Project [codegen id : 3] -Output [6]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Input [8]: [ws_sold_date_sk#4, ws_item_sk#5, ws_web_page_sk#6, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wp_web_page_sk#11] +(9) Project [codegen id : 2] +Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Input [8]: [ws_sold_date_sk#1, ws_item_sk#2, ws_web_page_sk#3, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wp_web_page_sk#8] -(15) Exchange -Input [6]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Arguments: hashpartitioning(cast(ws_item_sk#5 as bigint), cast(ws_order_number#7 as bigint), 5), true, [id=#13] +(10) Exchange +Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Arguments: hashpartitioning(cast(ws_item_sk#2 as bigint), cast(ws_order_number#4 as bigint), 5), true, [id=#10] -(16) Sort [codegen id : 4] -Input [6]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10] -Arguments: [cast(ws_item_sk#5 as bigint) ASC NULLS FIRST, cast(ws_order_number#7 as bigint) ASC NULLS FIRST], false, 0 +(11) Sort [codegen id : 3] +Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7] +Arguments: [cast(ws_item_sk#2 as bigint) ASC NULLS FIRST, cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 -(17) Scan parquet default.web_returns -Output [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] +(12) Scan parquet default.web_returns +Output [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 5] -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] +(13) ColumnarToRow [codegen id : 4] +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] -(19) Filter [codegen id : 5] -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] -Condition : (((((isnotnull(wr_item_sk#14) AND isnotnull(wr_order_number#19)) AND isnotnull(wr_refunded_cdemo_sk#15)) AND isnotnull(wr_returning_cdemo_sk#17)) AND isnotnull(wr_refunded_addr_sk#16)) AND isnotnull(wr_reason_sk#18)) +(14) Filter [codegen id : 4] +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] +Condition : (((((isnotnull(wr_item_sk#11) AND isnotnull(wr_order_number#16)) AND isnotnull(wr_refunded_cdemo_sk#12)) AND isnotnull(wr_returning_cdemo_sk#14)) AND isnotnull(wr_refunded_addr_sk#13)) AND isnotnull(wr_reason_sk#15)) -(20) Exchange -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] -Arguments: hashpartitioning(wr_item_sk#14, wr_order_number#19, 5), true, [id=#22] +(15) Exchange +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] +Arguments: hashpartitioning(wr_item_sk#11, wr_order_number#16, 5), true, [id=#19] -(21) Sort [codegen id : 6] -Input [8]: [wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] -Arguments: [wr_item_sk#14 ASC NULLS FIRST, wr_order_number#19 ASC NULLS FIRST], false, 0 +(16) Sort [codegen id : 5] +Input [8]: [wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] +Arguments: [wr_item_sk#11 ASC NULLS FIRST, wr_order_number#16 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#5 as bigint), cast(ws_order_number#7 as bigint)] -Right keys [2]: [wr_item_sk#14, wr_order_number#19] +(17) SortMergeJoin [codegen id : 9] +Left keys [2]: [cast(ws_item_sk#2 as bigint), cast(ws_order_number#4 as bigint)] +Right keys [2]: [wr_item_sk#11, wr_order_number#16] Join condition: None -(23) Project -Output [10]: [ws_sold_date_sk#4, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21] -Input [14]: [ws_sold_date_sk#4, ws_item_sk#5, ws_order_number#7, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_item_sk#14, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_order_number#19, wr_fee#20, wr_refunded_cash#21] +(18) Project [codegen id : 9] +Output [10]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18] +Input [14]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#4, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_item_sk#11, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_order_number#16, wr_fee#17, wr_refunded_cash#18] + +(19) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_year#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(20) ColumnarToRow [codegen id : 6] +Input [2]: [d_date_sk#20, d_year#21] + +(21) Filter [codegen id : 6] +Input [2]: [d_date_sk#20, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2000)) AND isnotnull(d_date_sk#20)) + +(22) Project [codegen id : 6] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_year#21] + +(23) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ws_sold_date_sk#4] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#20] Join condition: None (25) Project [codegen id : 9] -Output [9]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21] -Input [11]: [d_date_sk#1, ws_sold_date_sk#4, ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21] +Output [9]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18] +Input [11]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18, d_date_sk#20] (26) Scan parquet default.reason Output [2]: [r_reason_sk#23, r_reason_desc#24] @@ -188,13 +188,13 @@ Input [2]: [r_reason_sk#23, r_reason_desc#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [wr_reason_sk#18] +Left keys [1]: [wr_reason_sk#15] Right keys [1]: [cast(r_reason_sk#23 as bigint)] Join condition: None (31) Project [codegen id : 9] -Output [9]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Input [11]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_reason_sk#18, wr_fee#20, wr_refunded_cash#21, r_reason_sk#23, r_reason_desc#24] +Output [9]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Input [11]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_reason_sk#15, wr_fee#17, wr_refunded_cash#18, r_reason_sk#23, r_reason_desc#24] (32) Scan parquet default.customer_address Output [3]: [ca_address_sk#26, ca_state#27, ca_country#28] @@ -219,84 +219,84 @@ Input [2]: [ca_address_sk#26, ca_state#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [wr_refunded_addr_sk#16] +Left keys [1]: [wr_refunded_addr_sk#13] Right keys [1]: [cast(ca_address_sk#26 as bigint)] -Join condition: ((((ca_state#27 IN (IN,OH,NJ) AND (ws_net_profit#10 >= 100.00)) AND (ws_net_profit#10 <= 200.00)) OR ((ca_state#27 IN (WI,CT,KY) AND (ws_net_profit#10 >= 150.00)) AND (ws_net_profit#10 <= 300.00))) OR ((ca_state#27 IN (LA,IA,AR) AND (ws_net_profit#10 >= 50.00)) AND (ws_net_profit#10 <= 250.00))) +Join condition: ((((ca_state#27 IN (IN,OH,NJ) AND (ws_net_profit#7 >= 100.00)) AND (ws_net_profit#7 <= 200.00)) OR ((ca_state#27 IN (WI,CT,KY) AND (ws_net_profit#7 >= 150.00)) AND (ws_net_profit#7 <= 300.00))) OR ((ca_state#27 IN (LA,IA,AR) AND (ws_net_profit#7 >= 50.00)) AND (ws_net_profit#7 <= 250.00))) (38) Project [codegen id : 9] -Output [7]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Input [11]: [ws_quantity#8, ws_sales_price#9, ws_net_profit#10, wr_refunded_cdemo_sk#15, wr_refunded_addr_sk#16, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24, ca_address_sk#26, ca_state#27] +Output [7]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Input [11]: [ws_quantity#5, ws_sales_price#6, ws_net_profit#7, wr_refunded_cdemo_sk#12, wr_refunded_addr_sk#13, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24, ca_address_sk#26, ca_state#27] (39) Exchange -Input [7]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Arguments: hashpartitioning(wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, 5), true, [id=#30] +Input [7]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Arguments: hashpartitioning(wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, 5), true, [id=#30] (40) Sort [codegen id : 10] -Input [7]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Arguments: [wr_refunded_cdemo_sk#15 ASC NULLS FIRST, wr_returning_cdemo_sk#17 ASC NULLS FIRST], false, 0 +Input [7]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Arguments: [wr_refunded_cdemo_sk#12 ASC NULLS FIRST, wr_returning_cdemo_sk#14 ASC NULLS FIRST], false, 0 (41) Scan parquet default.customer_demographics Output [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct -(42) ColumnarToRow [codegen id : 12] +(42) ColumnarToRow [codegen id : 11] +Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] + +(43) Filter [codegen id : 11] Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] +Condition : (((isnotnull(cd_demo_sk#31) AND isnotnull(cd_marital_status#32)) AND isnotnull(cd_education_status#33)) AND ((((cd_marital_status#32 = M) AND (cd_education_status#33 = Advanced Degree)) OR ((cd_marital_status#32 = S) AND (cd_education_status#33 = College))) OR ((cd_marital_status#32 = W) AND (cd_education_status#33 = 2 yr Degree)))) -(43) Filter [codegen id : 12] +(44) BroadcastExchange Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] -Condition : ((isnotnull(cd_demo_sk#31) AND isnotnull(cd_marital_status#32)) AND isnotnull(cd_education_status#33)) +Arguments: HashedRelationBroadcastMode(List(input[1, string, false], input[2, string, false]),false), [id=#34] -(44) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] +(45) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 11] -Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] - -(46) Filter [codegen id : 11] -Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Condition : (((isnotnull(cd_demo_sk#34) AND isnotnull(cd_marital_status#35)) AND isnotnull(cd_education_status#36)) AND ((((cd_marital_status#35 = M) AND (cd_education_status#36 = Advanced Degree)) OR ((cd_marital_status#35 = S) AND (cd_education_status#36 = College))) OR ((cd_marital_status#35 = W) AND (cd_education_status#36 = 2 yr Degree)))) +(46) ColumnarToRow +Input [3]: [cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] -(47) BroadcastExchange -Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: HashedRelationBroadcastMode(List(input[1, string, false], input[2, string, false]),false), [id=#37] +(47) Filter +Input [3]: [cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] +Condition : ((isnotnull(cd_demo_sk#35) AND isnotnull(cd_marital_status#36)) AND isnotnull(cd_education_status#37)) (48) BroadcastHashJoin [codegen id : 12] Left keys [2]: [cd_marital_status#32, cd_education_status#33] -Right keys [2]: [cd_marital_status#35, cd_education_status#36] +Right keys [2]: [cd_marital_status#36, cd_education_status#37] Join condition: None (49) Project [codegen id : 12] -Output [4]: [cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Input [6]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] +Output [4]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] +Input [6]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35, cd_marital_status#36, cd_education_status#37] (50) Exchange -Input [4]: [cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: hashpartitioning(cast(cd_demo_sk#34 as bigint), cast(cd_demo_sk#31 as bigint), 5), true, [id=#38] +Input [4]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] +Arguments: hashpartitioning(cast(cd_demo_sk#31 as bigint), cast(cd_demo_sk#35 as bigint), 5), true, [id=#38] (51) Sort [codegen id : 13] -Input [4]: [cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: [cast(cd_demo_sk#34 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#31 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] +Arguments: [cast(cd_demo_sk#31 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#35 as bigint) ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 14] -Left keys [2]: [wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17] -Right keys [2]: [cast(cd_demo_sk#34 as bigint), cast(cd_demo_sk#31 as bigint)] -Join condition: ((((((cd_marital_status#35 = M) AND (cd_education_status#36 = Advanced Degree)) AND (ws_sales_price#9 >= 100.00)) AND (ws_sales_price#9 <= 150.00)) OR ((((cd_marital_status#35 = S) AND (cd_education_status#36 = College)) AND (ws_sales_price#9 >= 50.00)) AND (ws_sales_price#9 <= 100.00))) OR ((((cd_marital_status#35 = W) AND (cd_education_status#36 = 2 yr Degree)) AND (ws_sales_price#9 >= 150.00)) AND (ws_sales_price#9 <= 200.00))) +Left keys [2]: [wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14] +Right keys [2]: [cast(cd_demo_sk#31 as bigint), cast(cd_demo_sk#35 as bigint)] +Join condition: ((((((cd_marital_status#32 = M) AND (cd_education_status#33 = Advanced Degree)) AND (ws_sales_price#6 >= 100.00)) AND (ws_sales_price#6 <= 150.00)) OR ((((cd_marital_status#32 = S) AND (cd_education_status#33 = College)) AND (ws_sales_price#6 >= 50.00)) AND (ws_sales_price#6 <= 100.00))) OR ((((cd_marital_status#32 = W) AND (cd_education_status#33 = 2 yr Degree)) AND (ws_sales_price#6 >= 150.00)) AND (ws_sales_price#6 <= 200.00))) (53) Project [codegen id : 14] -Output [4]: [ws_quantity#8, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] -Input [11]: [ws_quantity#8, ws_sales_price#9, wr_refunded_cdemo_sk#15, wr_returning_cdemo_sk#17, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24, cd_demo_sk#31, cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] +Output [4]: [ws_quantity#5, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] +Input [11]: [ws_quantity#5, ws_sales_price#6, wr_refunded_cdemo_sk#12, wr_returning_cdemo_sk#14, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24, cd_demo_sk#31, cd_marital_status#32, cd_education_status#33, cd_demo_sk#35] (54) HashAggregate [codegen id : 14] -Input [4]: [ws_quantity#8, wr_fee#20, wr_refunded_cash#21, r_reason_desc#24] +Input [4]: [ws_quantity#5, wr_fee#17, wr_refunded_cash#18, r_reason_desc#24] Keys [1]: [r_reason_desc#24] -Functions [3]: [partial_avg(cast(ws_quantity#8 as bigint)), partial_avg(UnscaledValue(wr_refunded_cash#21)), partial_avg(UnscaledValue(wr_fee#20))] +Functions [3]: [partial_avg(cast(ws_quantity#5 as bigint)), partial_avg(UnscaledValue(wr_refunded_cash#18)), partial_avg(UnscaledValue(wr_fee#17))] Aggregate Attributes [6]: [sum#39, count#40, sum#41, count#42, sum#43, count#44] Results [7]: [r_reason_desc#24, sum#45, count#46, sum#47, count#48, sum#49, count#50] @@ -307,9 +307,9 @@ Arguments: hashpartitioning(r_reason_desc#24, 5), true, [id=#51] (56) HashAggregate [codegen id : 15] Input [7]: [r_reason_desc#24, sum#45, count#46, sum#47, count#48, sum#49, count#50] Keys [1]: [r_reason_desc#24] -Functions [3]: [avg(cast(ws_quantity#8 as bigint)), avg(UnscaledValue(wr_refunded_cash#21)), avg(UnscaledValue(wr_fee#20))] -Aggregate Attributes [3]: [avg(cast(ws_quantity#8 as bigint))#52, avg(UnscaledValue(wr_refunded_cash#21))#53, avg(UnscaledValue(wr_fee#20))#54] -Results [5]: [substr(r_reason_desc#24, 1, 20) AS substr(r_reason_desc, 1, 20)#55, avg(cast(ws_quantity#8 as bigint))#52 AS avg(ws_quantity)#56, cast((avg(UnscaledValue(wr_refunded_cash#21))#53 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#57, cast((avg(UnscaledValue(wr_fee#20))#54 / 100.0) as decimal(11,6)) AS avg(wr_fee)#58, avg(cast(ws_quantity#8 as bigint))#52 AS aggOrder#59] +Functions [3]: [avg(cast(ws_quantity#5 as bigint)), avg(UnscaledValue(wr_refunded_cash#18)), avg(UnscaledValue(wr_fee#17))] +Aggregate Attributes [3]: [avg(cast(ws_quantity#5 as bigint))#52, avg(UnscaledValue(wr_refunded_cash#18))#53, avg(UnscaledValue(wr_fee#17))#54] +Results [5]: [substr(r_reason_desc#24, 1, 20) AS substr(r_reason_desc, 1, 20)#55, avg(cast(ws_quantity#5 as bigint))#52 AS avg(ws_quantity)#56, cast((avg(UnscaledValue(wr_refunded_cash#18))#53 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#57, cast((avg(UnscaledValue(wr_fee#17))#54 / 100.0) as decimal(11,6)) AS avg(wr_fee)#58, avg(cast(ws_quantity#5 as bigint))#52 AS aggOrder#59] (57) TakeOrderedAndProject Input [5]: [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58, aggOrder#59] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt index e7aee17172e6..3fa7d84f5596 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt @@ -18,23 +18,15 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas Project [ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_fee,wr_refunded_cash,r_reason_desc] BroadcastHashJoin [wr_reason_sk,r_reason_sk] Project [ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [ws_item_sk,ws_order_number] InputAdapter - Exchange [ws_item_sk,ws_order_number] #4 - WholeStageCodegen (3) + Exchange [ws_item_sk,ws_order_number] #3 + WholeStageCodegen (2) Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sold_date_sk,ws_sales_price,ws_net_profit] @@ -42,22 +34,30 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) + BroadcastExchange #4 + WholeStageCodegen (1) Filter [wp_web_page_sk] ColumnarToRow InputAdapter Scan parquet default.web_page [wp_web_page_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (5) Sort [wr_item_sk,wr_order_number] InputAdapter - Exchange [wr_item_sk,wr_order_number] #6 - WholeStageCodegen (5) + Exchange [wr_item_sk,wr_order_number] #5 + WholeStageCodegen (4) Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) @@ -79,12 +79,8 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas InputAdapter Exchange [cd_demo_sk,cd_demo_sk] #9 WholeStageCodegen (12) - Project [cd_demo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + Project [cd_demo_sk,cd_marital_status,cd_education_status,cd_demo_sk] BroadcastHashJoin [cd_marital_status,cd_education_status,cd_marital_status,cd_education_status] - Filter [cd_demo_sk,cd_marital_status,cd_education_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #10 WholeStageCodegen (11) @@ -92,3 +88,7 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cas ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] 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 69b02557c475..4e85516b594f 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 @@ -13,14 +13,14 @@ : : :- * Project (16) : : : +- * BroadcastHashJoin Inner BuildRight (15) : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildLeft (8) - : : : : :- BroadcastExchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer_demographics (1) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet default.customer (5) + : : : : +- * 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) @@ -33,61 +33,61 @@ : : +- Scan parquet default.customer_address (17) : +- BroadcastExchange (34) : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildLeft (32) - : :- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.date_dim (24) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.catalog_returns (29) + : +- * 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.customer_demographics -Output [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +(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}/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 - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] +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 -(3) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Condition : ((((cd_marital_status#2 = M) AND (cd_education_status#3 = Unknown)) OR ((cd_marital_status#2 = W) AND (cd_education_status#3 = Advanced Degree))) AND isnotnull(cd_demo_sk#1)) +(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] -(4) BroadcastExchange -Input [3]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#4] +(3) Filter [codegen id : 7] +Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#4)) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) -(5) Scan parquet default.customer -Output [4]: [c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/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}/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]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -(6) ColumnarToRow -Input [4]: [c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] +(6) Filter [codegen id : 1] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Condition : ((((cd_marital_status#6 = M) AND (cd_education_status#7 = Unknown)) OR ((cd_marital_status#6 = W) AND (cd_education_status#7 = Advanced Degree))) AND isnotnull(cd_demo_sk#5)) -(7) Filter -Input [4]: [c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] -Condition : (((isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#8)) AND isnotnull(c_current_cdemo_sk#6)) AND isnotnull(c_current_hdemo_sk#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) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cd_demo_sk#1] -Right keys [1]: [c_current_cdemo_sk#6] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#5] Join condition: None (9) Project [codegen id : 7] -Output [5]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_hdemo_sk#7, c_current_addr_sk#8] -Input [7]: [cd_demo_sk#1, cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_cdemo_sk#6, c_current_hdemo_sk#7, c_current_addr_sk#8] +Output [5]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] +Input [7]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] (10) Scan parquet default.household_demographics Output [2]: [hd_demo_sk#9, hd_buy_potential#10] @@ -112,13 +112,13 @@ 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]: [c_current_hdemo_sk#7] +Left keys [1]: [c_current_hdemo_sk#3] Right keys [1]: [hd_demo_sk#9] Join condition: None (16) Project [codegen id : 7] -Output [4]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_addr_sk#8] -Input [6]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_hdemo_sk#7, c_current_addr_sk#8, hd_demo_sk#9] +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.customer_address Output [2]: [ca_address_sk#12, ca_gmt_offset#13] @@ -143,71 +143,71 @@ 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_addr_sk#8] +Left keys [1]: [c_current_addr_sk#4] Right keys [1]: [ca_address_sk#12] Join condition: None (23) Project [codegen id : 7] -Output [3]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5] -Input [5]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, c_current_addr_sk#8, ca_address_sk#12] +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.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] +(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}/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}/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 [3]: [d_date_sk#15, d_year#16, d_moy#17] +(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 [3]: [d_date_sk#15, d_year#16, d_moy#17] -Condition : ((((isnotnull(d_year#16) AND isnotnull(d_moy#17)) AND (d_year#16 = 1998)) AND (d_moy#17 = 11)) AND isnotnull(d_date_sk#15)) +(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]: [d_date_sk#15] -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] +(27) Scan parquet default.date_dim +Output [3]: [d_date_sk#19, d_year#20, d_moy#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct -(28) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +(28) ColumnarToRow [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(29) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] -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 +(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) ColumnarToRow -Input [4]: [cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +(30) Project [codegen id : 4] +Output [1]: [d_date_sk#19] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(31) Filter -Input [4]: [cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] -Condition : ((isnotnull(cr_call_center_sk#21) AND isnotnull(cr_returned_date_sk#19)) AND isnotnull(cr_returning_customer_sk#20)) +(31) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#15] -Right keys [1]: [cr_returned_date_sk#19] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] Join condition: None (33) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] -Input [5]: [d_date_sk#15, cr_returned_date_sk#19, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +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]: [cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#22] +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_customer_sk#5] -Right keys [1]: [cr_returning_customer_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]: [cd_marital_status#2, cd_education_status#3, cr_call_center_sk#21, cr_net_loss#22] -Input [6]: [cd_marital_status#2, cd_education_status#3, c_customer_sk#5, cr_returning_customer_sk#20, cr_call_center_sk#21, cr_net_loss#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#21] +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#22, cd_marital_status#2, cd_education_status#3] -Input [8]: [cd_marital_status#2, cd_education_status#3, cr_call_center_sk#21, cr_net_loss#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#22, cd_marital_status#2, cd_education_status#3] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#22))] +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#2, cd_education_status#3, 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#2, cd_education_status#3, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3, 5), true, [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), true, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#2, cd_education_status#3] -Functions [1]: [sum(UnscaledValue(cr_net_loss#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#22))#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#22))#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 f64791821893..87beb3b565cc 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 @@ -12,12 +12,16 @@ WholeStageCodegen (9) BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [cd_marital_status,cd_education_status,c_customer_sk] + Project [c_customer_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cd_marital_status,cd_education_status,c_customer_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 [cd_marital_status,cd_education_status,c_customer_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cd_demo_sk,c_current_cdemo_sk] + Project [c_customer_sk,c_current_hdemo_sk,c_current_addr_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + 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) @@ -25,10 +29,6 @@ WholeStageCodegen (9) ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - 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 #4 WholeStageCodegen (2) @@ -49,7 +49,11 @@ WholeStageCodegen (9) BroadcastExchange #6 WholeStageCodegen (5) Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [d_date_sk,cr_returned_date_sk] + 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 #7 WholeStageCodegen (4) @@ -58,10 +62,6 @@ WholeStageCodegen (9) ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - 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 #8 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt index c547e7af5d79..34eba382992c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject (32) : :- * 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.catalog_sales (6) + : : : +- * 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) : : +- BroadcastExchange (14) : : +- * Filter (13) : : +- * ColumnarToRow (12) @@ -33,50 +33,50 @@ TakeOrderedAndProject (32) +- Scan parquet default.warehouse (23) -(1) Scan parquet default.date_dim -Output [2]: [d_date_sk#1, d_month_seq#2] +(1) Scan parquet default.catalog_sales +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(2) ColumnarToRow [codegen id : 5] +Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] -(3) Filter [codegen id : 1] -Input [2]: [d_date_sk#1, d_month_seq#2] -Condition : (((isnotnull(d_month_seq#2) AND (d_month_seq#2 >= 1200)) AND (d_month_seq#2 <= 1211)) AND isnotnull(d_date_sk#1)) +(3) Filter [codegen id : 5] +Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] +Condition : (((isnotnull(cs_warehouse_sk#5) AND isnotnull(cs_ship_mode_sk#4)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [2]: [d_date_sk#1, d_month_seq#2] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] -(6) Scan parquet default.catalog_sales -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct +(6) Filter [codegen id : 1] +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(7) ColumnarToRow -Input [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#6] +Input [2]: [d_date_sk#6, d_month_seq#7] -(8) Filter -Input [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] -Condition : (((isnotnull(cs_warehouse_sk#8) AND isnotnull(cs_ship_mode_sk#7)) AND isnotnull(cs_call_center_sk#6)) AND isnotnull(cs_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [cs_ship_date_sk#5] +Left keys [1]: [cs_ship_date_sk#2] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] -Input [6]: [d_date_sk#1, cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5] +Input [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5, d_date_sk#6] (11) Scan parquet default.ship_mode Output [2]: [sm_ship_mode_sk#9, sm_type#10] @@ -97,13 +97,13 @@ Input [2]: [sm_ship_mode_sk#9, sm_type#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_mode_sk#7] +Left keys [1]: [cs_ship_mode_sk#4] Right keys [1]: [sm_ship_mode_sk#9] Join condition: None (16) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_warehouse_sk#8, sm_type#10] -Input [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_ship_mode_sk#7, cs_warehouse_sk#8, sm_ship_mode_sk#9, sm_type#10] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_warehouse_sk#5, sm_type#10] +Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_ship_mode_sk#4, cs_warehouse_sk#5, sm_ship_mode_sk#9, sm_type#10] (17) Scan parquet default.call_center Output [2]: [cc_call_center_sk#12, cc_name#13] @@ -124,13 +124,13 @@ Input [2]: [cc_call_center_sk#12, cc_name#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_call_center_sk#6] +Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#12] Join condition: None (22) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_warehouse_sk#8, sm_type#10, cc_name#13] -Input [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_call_center_sk#6, cs_warehouse_sk#8, sm_type#10, cc_call_center_sk#12, cc_name#13] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_warehouse_sk#5, sm_type#10, cc_name#13] +Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_call_center_sk#3, cs_warehouse_sk#5, sm_type#10, cc_call_center_sk#12, cc_name#13] (23) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#15, w_warehouse_name#16] @@ -151,18 +151,18 @@ Input [2]: [w_warehouse_sk#15, w_warehouse_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_warehouse_sk#8] +Left keys [1]: [cs_warehouse_sk#5] Right keys [1]: [w_warehouse_sk#15] Join condition: None (28) Project [codegen id : 5] -Output [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, w_warehouse_name#16, sm_type#10, cc_name#13] -Input [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_warehouse_sk#8, sm_type#10, cc_name#13, w_warehouse_sk#15, w_warehouse_name#16] +Output [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#16, sm_type#10, cc_name#13] +Input [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_warehouse_sk#5, sm_type#10, cc_name#13, w_warehouse_sk#15, w_warehouse_name#16] (29) HashAggregate [codegen id : 5] -Input [5]: [cs_sold_date_sk#4, cs_ship_date_sk#5, w_warehouse_name#16, sm_type#10, cc_name#13] +Input [5]: [cs_sold_date_sk#1, cs_ship_date_sk#2, w_warehouse_name#16, sm_type#10, cc_name#13] Keys [3]: [substr(w_warehouse_name#16, 1, 20) AS substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13] -Functions [5]: [partial_sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] +Functions [5]: [partial_sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), partial_sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] Aggregate Attributes [5]: [sum#19, sum#20, sum#21, sum#22, sum#23] Results [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] @@ -173,9 +173,9 @@ Arguments: hashpartitioning(substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, c (31) HashAggregate [codegen id : 6] Input [8]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13, sum#24, sum#25, sum#26, sum#27, sum#28] Keys [3]: [substr(w_warehouse_name#16, 1, 20)#18, sm_type#10, cc_name#13] -Functions [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))] -Aggregate Attributes [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34] -Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 30) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 60) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 90) AND ((cs_ship_date_sk#5 - cs_sold_date_sk#4) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((cs_ship_date_sk#5 - cs_sold_date_sk#4) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] +Functions [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint)), sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))] +Aggregate Attributes [5]: [sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34] +Results [8]: [substr(w_warehouse_name#16, 1, 20)#18 AS substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 30) THEN 1 ELSE 0 END as bigint))#30 AS 30 days #36, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 30) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 60)) THEN 1 ELSE 0 END as bigint))#31 AS 31 - 60 days #37, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 60) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 90)) THEN 1 ELSE 0 END as bigint))#32 AS 61 - 90 days #38, sum(cast(CASE WHEN (((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 90) AND ((cs_ship_date_sk#2 - cs_sold_date_sk#1) <= 120)) THEN 1 ELSE 0 END as bigint))#33 AS 91 - 120 days #39, sum(cast(CASE WHEN ((cs_ship_date_sk#2 - cs_sold_date_sk#1) > 120) THEN 1 ELSE 0 END as bigint))#34 AS >120 days #40] (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt index de3b1913ae25..b25b16136992 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt @@ -12,7 +12,11 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_warehouse_sk,sm_type] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] - BroadcastHashJoin [d_date_sk,cs_ship_date_sk] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -21,10 +25,6 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] - Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index ab246a344955..1b9e8f37e941 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -11,30 +11,30 @@ TakeOrderedAndProject (50) : +- Exchange (27) : +- * Project (26) : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) : : :- * 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 (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * BroadcastHashJoin LeftOuter BuildRight (14) + : : :- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) : +- BroadcastExchange (24) : +- * Project (23) : +- * Filter (22) - : +- * BroadcastHashJoin LeftOuter BuildRight (21) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.item (11) - : +- BroadcastExchange (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Filter (16) - : +- * ColumnarToRow (15) - : +- Scan parquet default.item (14) + : +- * ColumnarToRow (21) + : +- Scan parquet default.date_dim (20) +- * Sort (42) +- Exchange (41) +- * Project (40) @@ -65,112 +65,112 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3] Condition : ((isnotnull(ss_customer_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_item_sk#2)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_month_seq#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_month_seq#5] -Condition : ((isnotnull(d_month_seq#5) AND (d_month_seq#5 = Subquery scalar-subquery#6, [id=#7])) AND isnotnull(d_date_sk#4)) - -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_month_seq#5] - -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None - -(10) Project [codegen id : 5] -Output [2]: [ss_item_sk#2, ss_customer_sk#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, d_date_sk#4] - -(11) Scan parquet default.item -Output [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(4) Scan parquet default.item +Output [3]: [i_item_sk#4, i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] +(5) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] -(13) Filter [codegen id : 4] -Input [3]: [i_item_sk#9, i_current_price#10, i_category#11] -Condition : (isnotnull(i_current_price#10) AND isnotnull(i_item_sk#9)) +(6) Filter [codegen id : 3] +Input [3]: [i_item_sk#4, i_current_price#5, i_category#6] +Condition : (isnotnull(i_current_price#5) AND isnotnull(i_item_sk#4)) -(14) Scan parquet default.item -Output [2]: [i_current_price#10, i_category#11] +(7) Scan parquet default.item +Output [2]: [i_current_price#5, i_category#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] - -(16) Filter [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Condition : isnotnull(i_category#11) - -(17) HashAggregate [codegen id : 2] -Input [2]: [i_current_price#10, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [i_category#11, sum#14, count#15] - -(18) Exchange -Input [3]: [i_category#11, sum#14, count#15] -Arguments: hashpartitioning(i_category#11, 5), true, [id=#16] - -(19) HashAggregate [codegen id : 3] -Input [3]: [i_category#11, sum#14, count#15] -Keys [1]: [i_category#11] -Functions [1]: [avg(UnscaledValue(i_current_price#10))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#10))#17] -Results [2]: [cast((avg(UnscaledValue(i_current_price#10))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#11 AS i_category#11#19] - -(20) BroadcastExchange -Input [2]: [avg(i_current_price)#18, i_category#11#19] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#20] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_category#11] -Right keys [1]: [i_category#11#19] +(8) ColumnarToRow [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] + +(9) Filter [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Condition : isnotnull(i_category#6) + +(10) HashAggregate [codegen id : 1] +Input [2]: [i_current_price#5, i_category#6] +Keys [1]: [i_category#6] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [2]: [sum#7, count#8] +Results [3]: [i_category#6, sum#9, count#10] + +(11) Exchange +Input [3]: [i_category#6, sum#9, count#10] +Arguments: hashpartitioning(i_category#6, 5), true, [id=#11] + +(12) HashAggregate [codegen id : 2] +Input [3]: [i_category#6, sum#9, count#10] +Keys [1]: [i_category#6] +Functions [1]: [avg(UnscaledValue(i_current_price#5))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#5))#12] +Results [2]: [cast((avg(UnscaledValue(i_current_price#5))#12 / 100.0) as decimal(11,6)) AS avg(i_current_price)#13, i_category#6 AS i_category#6#14] + +(13) BroadcastExchange +Input [2]: [avg(i_current_price)#13, i_category#6#14] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#15] + +(14) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#6] +Right keys [1]: [i_category#6#14] +Join condition: None + +(15) Filter [codegen id : 3] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] +Condition : (cast(i_current_price#5 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#13)), DecimalType(14,7), true)) + +(16) Project [codegen id : 3] +Output [1]: [i_item_sk#4] +Input [5]: [i_item_sk#4, i_current_price#5, i_category#6, avg(i_current_price)#13, i_category#6#14] + +(17) BroadcastExchange +Input [1]: [i_item_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None +(19) Project [codegen id : 5] +Output [2]: [ss_sold_date_sk#1, ss_customer_sk#3] +Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, i_item_sk#4] + +(20) Scan parquet default.date_dim +Output [2]: [d_date_sk#17, d_month_seq#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(21) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#17, d_month_seq#18] + (22) Filter [codegen id : 4] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] -Condition : (cast(i_current_price#10 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#18)), DecimalType(14,7), true)) +Input [2]: [d_date_sk#17, d_month_seq#18] +Condition : ((isnotnull(d_month_seq#18) AND (d_month_seq#18 = Subquery scalar-subquery#19, [id=#20])) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 4] -Output [1]: [i_item_sk#9] -Input [5]: [i_item_sk#9, i_current_price#10, i_category#11, avg(i_current_price)#18, i_category#11#19] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_month_seq#18] (24) BroadcastExchange -Input [1]: [i_item_sk#9] +Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#3] -Input [3]: [ss_item_sk#2, ss_customer_sk#3, i_item_sk#9] +Input [3]: [ss_sold_date_sk#1, ss_customer_sk#3, d_date_sk#17] (27) Exchange Input [1]: [ss_customer_sk#3] @@ -282,7 +282,7 @@ Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#24 ASC NULLS FIRST], [state#34 ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#6, [id=#7] +Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#19, [id=#20] * HashAggregate (57) +- Exchange (56) +- * HashAggregate (55) @@ -293,39 +293,39 @@ Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery (51) Scan parquet default.date_dim -Output [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [3]: [d_month_seq#18, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (52) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (53) Filter [codegen id : 1] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] Condition : (((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2000)) AND (d_moy#38 = 1)) (54) Project [codegen id : 1] -Output [1]: [d_month_seq#5] -Input [3]: [d_month_seq#5, d_year#37, d_moy#38] +Output [1]: [d_month_seq#18] +Input [3]: [d_month_seq#18, d_year#37, d_moy#38] (55) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] (56) Exchange -Input [1]: [d_month_seq#5] -Arguments: hashpartitioning(d_month_seq#5, 5), true, [id=#39] +Input [1]: [d_month_seq#18] +Arguments: hashpartitioning(d_month_seq#18, 5), true, [id=#39] (57) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#5] -Keys [1]: [d_month_seq#5] +Input [1]: [d_month_seq#18] +Keys [1]: [d_month_seq#18] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#5] +Results [1]: [d_month_seq#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index 2700741b82c0..3cbd44fc5a7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -16,55 +16,55 @@ TakeOrderedAndProject [cnt,ca_state,state] Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Project [ss_customer_sk] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_customer_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [d_month_seq] + WholeStageCodegen (3) + Project [i_item_sk] + Filter [i_current_price,avg(i_current_price)] + BroadcastHashJoin [i_category,i_category] + Filter [i_current_price,i_item_sk] + ColumnarToRow InputAdapter - Exchange [d_month_seq] #4 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] + BroadcastExchange #4 + WholeStageCodegen (2) + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + InputAdapter + Exchange [i_category] #5 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_current_price,i_category] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - Project [i_item_sk] - Filter [i_current_price,avg(i_current_price)] - BroadcastHashJoin [i_category,i_category] - Filter [i_current_price,i_item_sk] - ColumnarToRow + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [d_month_seq] InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_category] + Exchange [d_month_seq] #7 + WholeStageCodegen (1) + HashAggregate [d_month_seq] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] - InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_current_price,i_category] + Scan parquet default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] 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 c2627bd7e4cc..a7f328537b7a 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 @@ -24,15 +24,15 @@ TakeOrderedAndProject (79) : : : : : :- * Project (17) : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : :- BroadcastExchange (5) - : : : : : : : : +- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.household_demographics (1) - : : : : : : : +- * Filter (8) - : : : : : : : +- * ColumnarToRow (7) - : : : : : : : +- Scan parquet default.catalog_sales (6) + : : : : : : : +- * 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) @@ -49,26 +49,26 @@ TakeOrderedAndProject (79) : : : : +- Scan parquet default.item (26) : : : +- BroadcastExchange (43) : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : :- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- BroadcastExchange (40) - : : : +- * Project (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.date_dim (36) + : : : +- * 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 BuildLeft (55) - : : :- BroadcastExchange (51) - : : : +- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.warehouse (48) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet default.inventory (52) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) + : : +- BroadcastExchange (54) + : : +- * Filter (53) + : : +- * ColumnarToRow (52) + : : +- Scan parquet default.warehouse (51) : +- BroadcastExchange (64) : +- * Filter (63) : +- * ColumnarToRow (62) @@ -80,50 +80,50 @@ TakeOrderedAndProject (79) +- Scan parquet default.catalog_returns (69) -(1) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#1, hd_buy_potential#2] +(1) Scan parquet default.catalog_sales +Output [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(2) ColumnarToRow [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] -(3) Filter [codegen id : 1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] -Condition : ((isnotnull(hd_buy_potential#2) AND (hd_buy_potential#2 = 1001-5000)) AND isnotnull(hd_demo_sk#1)) +(3) Filter [codegen id : 4] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnotnull(cs_bill_cdemo_sk#3)) AND isnotnull(cs_bill_hdemo_sk#4)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_ship_date_sk#2)) -(4) Project [codegen id : 1] -Output [1]: [hd_demo_sk#1] -Input [2]: [hd_demo_sk#1, hd_buy_potential#2] +(4) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#9, hd_buy_potential#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +ReadSchema: struct -(5) BroadcastExchange -Input [1]: [hd_demo_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#3] +(5) ColumnarToRow [codegen id : 1] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(6) Scan parquet default.catalog_sales -Output [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct +(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)) -(7) ColumnarToRow -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +(7) Project [codegen id : 1] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(8) Filter -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Condition : (((((isnotnull(cs_quantity#11) AND isnotnull(cs_item_sk#8)) AND isnotnull(cs_bill_cdemo_sk#6)) AND isnotnull(cs_bill_hdemo_sk#7)) AND isnotnull(cs_sold_date_sk#4)) AND isnotnull(cs_ship_date_sk#5)) +(8) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [hd_demo_sk#1] -Right keys [1]: [cs_bill_hdemo_sk#7] +Left keys [1]: [cs_bill_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#9] Join condition: None (10) Project [codegen id : 4] -Output [7]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [9]: [hd_demo_sk#1, cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_bill_hdemo_sk#7, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] +Output [7]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [9]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_bill_hdemo_sk#4, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, hd_demo_sk#9] (11) Scan parquet default.customer_demographics Output [2]: [cd_demo_sk#12, cd_marital_status#13] @@ -148,13 +148,13 @@ Input [1]: [cd_demo_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#6] +Left keys [1]: [cs_bill_cdemo_sk#3] Right keys [1]: [cd_demo_sk#12] Join condition: None (17) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, cd_demo_sk#12] +Output [6]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, cd_demo_sk#12] (18) Scan parquet default.date_dim Output [2]: [d_date_sk#15, d_date#16] @@ -175,21 +175,21 @@ Input [2]: [d_date_sk#15, d_date#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_ship_date_sk#5] +Left keys [1]: [cs_ship_date_sk#2] Right keys [1]: [d_date_sk#15] Join condition: None (23) Project [codegen id : 4] -Output [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date_sk#15, d_date#16] +Output [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date_sk#15, d_date#16] (24) Exchange -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: hashpartitioning(cs_item_sk#8, 5), true, [id=#18] +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] (25) Sort [codegen id : 5] -Input [6]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16] -Arguments: [cs_item_sk#8 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] +Arguments: [cs_item_sk#5 ASC NULLS FIRST], false, 0 (26) Scan parquet default.item Output [2]: [i_item_sk#19, i_item_desc#20] @@ -214,137 +214,137 @@ Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 (31) SortMergeJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#8] +Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None (32) Project [codegen id : 10] -Output [7]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20] -Input [8]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_sk#19, i_item_desc#20] +Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_week_seq#23] +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_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct -(34) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(35) Filter [codegen id : 9] -Input [2]: [d_date_sk#22, d_week_seq#23] -Condition : (isnotnull(d_week_seq#23) AND isnotnull(d_date_sk#22)) +(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)) -(36) Scan parquet default.date_dim -Output [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -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) 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] -(37) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#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] -(38) Filter [codegen id : 8] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] -Condition : ((((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#24)) AND isnotnull(d_week_seq#26)) AND isnotnull(d_date#25)) +(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 -(39) Project [codegen id : 8] -Output [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Input [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(40) BroadcastExchange -Input [3]: [d_date_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#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)) (41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#23] -Right keys [1]: [d_week_seq#26] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None (42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] -Input [5]: [d_date_sk#22, d_week_seq#23, d_date_sk#24, d_date#25, d_week_seq#26] +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_sk#24, d_date#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#29] +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#4] -Right keys [1]: [d_date_sk#24] -Join condition: (d_date#16 > d_date#25 + 5 days) +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#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Input [11]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, d_date#16, i_item_desc#20, d_date_sk#22, d_date_sk#24, d_date#25, d_week_seq#26] +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] (46) Exchange -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, d_date_sk#22, 5), true, [id=#30] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] +Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] (47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, d_date_sk#22 ASC NULLS FIRST], false, 0 +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.warehouse -Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] +(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}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +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 -(50) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Condition : isnotnull(w_warehouse_sk#31) +(49) ColumnarToRow [codegen id : 13] +Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(51) BroadcastExchange -Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(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)) -(52) Scan parquet default.inventory -Output [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(51) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] 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 +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct + +(52) ColumnarToRow [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] -(53) ColumnarToRow -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +(53) Filter [codegen id : 12] +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Condition : isnotnull(w_warehouse_sk#35) -(54) Filter -Input [4]: [inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] -Condition : (((isnotnull(inv_quantity_on_hand#37) AND isnotnull(inv_item_sk#35)) AND isnotnull(inv_warehouse_sk#36)) AND isnotnull(inv_date_sk#34)) +(54) BroadcastExchange +Input [2]: [w_warehouse_sk#35, w_warehouse_name#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [w_warehouse_sk#31] -Right keys [1]: [inv_warehouse_sk#36] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Input [6]: [w_warehouse_sk#31, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_warehouse_sk#36, inv_quantity_on_hand#37] +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 [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: hashpartitioning(inv_item_sk#35, inv_date_sk#34, 5), true, [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), true, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] -Arguments: [inv_item_sk#35 ASC NULLS FIRST, inv_date_sk#34 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#8, d_date_sk#22] -Right keys [2]: [inv_item_sk#35, inv_date_sk#34] -Join condition: (inv_quantity_on_hand#37 < cs_quantity#11) +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#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [11]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, cs_quantity#11, i_item_desc#20, d_date_sk#22, d_week_seq#26, w_warehouse_name#32, inv_date_sk#34, inv_item_sk#35, inv_quantity_on_hand#37] +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] @@ -365,21 +365,21 @@ Input [1]: [p_promo_sk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (65) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [cs_promo_sk#9] +Left keys [1]: [cs_promo_sk#6] Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, 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#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: hashpartitioning(cs_item_sk#8, cs_order_number#10, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Arguments: [cs_item_sk#8 ASC NULLS FIRST, cs_order_number#10 ASC NULLS FIRST], false, 0 +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 Output [2]: [cr_item_sk#42, cr_order_number#43] @@ -404,33 +404,33 @@ Input [2]: [cr_item_sk#42, cr_order_number#43] Arguments: [cr_item_sk#42 ASC NULLS FIRST, cr_order_number#43 ASC NULLS FIRST], false, 0 (74) SortMergeJoin -Left keys [2]: [cs_item_sk#8, cs_order_number#10] +Left keys [2]: [cs_item_sk#5, cs_order_number#7] Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#32, i_item_desc#20, d_week_seq#26] -Input [7]: [cs_item_sk#8, cs_order_number#10, w_warehouse_name#32, i_item_desc#20, d_week_seq#26, 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#32, i_item_desc#20, d_week_seq#26] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +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#32, d_week_seq#26, 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#32, d_week_seq#26, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#32, d_week_seq#26, 5), true, [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), true, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#32, d_week_seq#26] +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#32, d_week_seq#26, 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#32, d_week_seq#26, 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#32 ASC NULLS FIRST, d_week_seq#26 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#32, d_week_seq#26, 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 39dba3af0235..918508787c4b 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 @@ -23,7 +23,7 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_date_sk,d_week_seq] + 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] @@ -38,7 +38,11 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom 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 [hd_demo_sk,cs_bill_hdemo_sk] + 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 BroadcastExchange #5 WholeStageCodegen (1) @@ -47,10 +51,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] - 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 BroadcastExchange #6 WholeStageCodegen (2) @@ -79,12 +79,8 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Project [d_date_sk,d_date_sk,d_date,d_week_seq] + Project [d_date_sk,d_date,d_week_seq,d_date_sk] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -93,14 +89,22 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom 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,inv_date_sk] InputAdapter Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [w_warehouse_name,inv_date_sk,inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [w_warehouse_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) @@ -108,10 +112,6 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #13 WholeStageCodegen (15) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index e6210f4a2628..025e5a6f9474 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -44,12 +44,12 @@ TakeOrderedAndProject (125) : : : : : +- * Project (23) : : : : : +- * Filter (22) : : : : : +- * ColumnarToRow (21) - : : : : : +- Scan parquet default.date_dim (20) + : : : : : +- Scan parquet default.promotion (20) : : : : +- BroadcastExchange (31) : : : : +- * Project (30) : : : : +- * Filter (29) : : : : +- * ColumnarToRow (28) - : : : : +- Scan parquet default.promotion (27) + : : : : +- Scan parquet default.date_dim (27) : : : +- BroadcastExchange (37) : : : +- * Filter (36) : : : +- * ColumnarToRow (35) @@ -210,67 +210,67 @@ Join condition: None Output [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, i_item_sk#14] -(20) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +(20) Scan parquet default.promotion +Output [2]: [p_promo_sk#17, p_channel_tv#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] +ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (22) Filter [codegen id : 6] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10442)) AND (d_date#18 <= 10472)) AND isnotnull(d_date_sk#17)) +Input [2]: [p_promo_sk#17, p_channel_tv#18] +Condition : ((isnotnull(p_channel_tv#18) AND (p_channel_tv#18 = N)) AND isnotnull(p_promo_sk#17)) (23) Project [codegen id : 6] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [p_promo_sk#17] +Input [2]: [p_promo_sk#17, p_channel_tv#18] (24) BroadcastExchange -Input [1]: [d_date_sk#17] +Input [1]: [p_promo_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_promo_sk#4] +Right keys [1]: [p_promo_sk#17] Join condition: None (26) Project [codegen id : 9] -Output [6]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#17] +Output [6]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] +Input [8]: [ss_sold_date_sk#1, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#17] -(27) Scan parquet default.promotion -Output [2]: [p_promo_sk#20, p_channel_tv#21] +(27) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_date#21] Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] +ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Input [2]: [d_date_sk#20, d_date#21] (29) Filter [codegen id : 7] -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Condition : ((isnotnull(p_channel_tv#21) AND (p_channel_tv#21 = N)) AND isnotnull(p_promo_sk#20)) +Input [2]: [d_date_sk#20, d_date#21] +Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 10442)) AND (d_date#21 <= 10472)) AND isnotnull(d_date_sk#20)) (30) Project [codegen id : 7] -Output [1]: [p_promo_sk#20] -Input [2]: [p_promo_sk#20, p_channel_tv#21] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_date#21] (31) BroadcastExchange -Input [1]: [p_promo_sk#20] +Input [1]: [d_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#20] Join condition: None (33) Project [codegen id : 9] Output [5]: [ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12] -Input [7]: [ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, p_promo_sk#20] +Input [7]: [ss_sold_date_sk#1, ss_store_sk#3, ss_ext_sales_price#6, ss_net_profit#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#20] (34) Scan parquet default.store Output [2]: [s_store_sk#23, s_store_id#24] @@ -383,28 +383,28 @@ Output [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_s Input [9]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_item_sk#47, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, i_item_sk#14] (58) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (59) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#45] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_promo_sk#48] +Right keys [1]: [p_promo_sk#17] Join condition: None (60) Project [codegen id : 19] -Output [6]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#17] +Output [6]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] +Input [8]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#17] (61) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (62) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_promo_sk#48] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [cs_sold_date_sk#45] +Right keys [1]: [d_date_sk#20] Join condition: None (63) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56] -Input [7]: [cs_catalog_page_sk#46, cs_promo_sk#48, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, p_promo_sk#20] +Input [7]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#50, cs_net_profit#51, cr_return_amount#55, cr_net_loss#56, d_date_sk#20] (64) Scan parquet default.catalog_page Output [2]: [cp_catalog_page_sk#58, cp_catalog_page_id#59] @@ -517,28 +517,28 @@ Output [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales Input [9]: [ws_sold_date_sk#80, ws_item_sk#81, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, i_item_sk#14] (88) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#17] +Output [1]: [p_promo_sk#17] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_promo_sk#83] +Right keys [1]: [p_promo_sk#17] Join condition: None (90) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#17] +Output [6]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] +Input [8]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#17] (91) ReusedExchange [Reuses operator id: 31] -Output [1]: [p_promo_sk#20] +Output [1]: [d_date_sk#20] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#83] -Right keys [1]: [p_promo_sk#20] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#20] Join condition: None (93) Project [codegen id : 29] Output [5]: [ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91] -Input [7]: [ws_web_site_sk#82, ws_promo_sk#83, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, p_promo_sk#20] +Input [7]: [ws_sold_date_sk#80, ws_web_site_sk#82, ws_ext_sales_price#85, ws_net_profit#86, wr_return_amt#90, wr_net_loss#91, d_date_sk#20] (94) Scan parquet default.web_site Output [2]: [web_site_sk#93, web_site_id#94] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt index 13781c8bd599..ad59968740aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt @@ -32,9 +32,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] Project [ss_sold_date_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] @@ -69,19 +69,19 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Project [d_date_sk] - Filter [d_date,d_date_sk] + Project [p_promo_sk] + Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) - Project [p_promo_sk] - Filter [p_channel_tv,p_promo_sk] + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_promo_sk,p_channel_tv] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -98,9 +98,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] @@ -127,9 +127,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #7 InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [p_promo_sk] #8 InputAdapter - ReusedExchange [p_promo_sk] #9 + ReusedExchange [d_date_sk] #9 InputAdapter BroadcastExchange #14 WholeStageCodegen (18) @@ -146,9 +146,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] Project [ws_sold_date_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] @@ -175,9 +175,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [i_item_sk] #7 InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [p_promo_sk] #8 InputAdapter - ReusedExchange [p_promo_sk] #9 + ReusedExchange [d_date_sk] #9 InputAdapter BroadcastExchange #18 WholeStageCodegen (28)