From d3ea308c8fe914085d38b96bbd32442fc8ebd3bd Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 14 Jan 2021 04:37:55 +0000 Subject: [PATCH] [SPARK-34081][SQL] Only pushdown LeftSemi/LeftAnti over Aggregate if join can be planned as broadcast join ### What changes were proposed in this pull request? Should not pushdown LeftSemi/LeftAnti over Aggregate for some cases. ```scala spark.range(50000000L).selectExpr("id % 10000 as a", "id % 10000 as b").write.saveAsTable("t1") spark.range(40000000L).selectExpr("id % 8000 as c", "id % 8000 as d").write.saveAsTable("t2") spark.sql("SELECT distinct a, b FROM t1 INTERSECT SELECT distinct c, d FROM t2").explain ``` Before this pr: ``` == Physical Plan == AdaptiveSparkPlan isFinalPlan=false +- HashAggregate(keys=[a#16L, b#17L], functions=[]) +- HashAggregate(keys=[a#16L, b#17L], functions=[]) +- HashAggregate(keys=[a#16L, b#17L], functions=[]) +- Exchange hashpartitioning(a#16L, b#17L, 5), ENSURE_REQUIREMENTS, [id=#72] +- HashAggregate(keys=[a#16L, b#17L], functions=[]) +- SortMergeJoin [coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L)], [coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L)], LeftSemi :- Sort [coalesce(a#16L, 0) ASC NULLS FIRST, isnull(a#16L) ASC NULLS FIRST, coalesce(b#17L, 0) ASC NULLS FIRST, isnull(b#17L) ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L), 5), ENSURE_REQUIREMENTS, [id=#65] : +- FileScan parquet default.t1[a#16L,b#17L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- Sort [coalesce(c#18L, 0) ASC NULLS FIRST, isnull(c#18L) ASC NULLS FIRST, coalesce(d#19L, 0) ASC NULLS FIRST, isnull(d#19L) ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L), 5), ENSURE_REQUIREMENTS, [id=#66] +- HashAggregate(keys=[c#18L, d#19L], functions=[]) +- Exchange hashpartitioning(c#18L, d#19L, 5), ENSURE_REQUIREMENTS, [id=#61] +- HashAggregate(keys=[c#18L, d#19L], functions=[]) +- FileScan parquet default.t2[c#18L,d#19L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` After this pr: ``` == Physical Plan == AdaptiveSparkPlan isFinalPlan=false +- HashAggregate(keys=[a#16L, b#17L], functions=[]) +- Exchange hashpartitioning(a#16L, b#17L, 5), ENSURE_REQUIREMENTS, [id=#74] +- HashAggregate(keys=[a#16L, b#17L], functions=[]) +- SortMergeJoin [coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L)], [coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L)], LeftSemi :- Sort [coalesce(a#16L, 0) ASC NULLS FIRST, isnull(a#16L) ASC NULLS FIRST, coalesce(b#17L, 0) ASC NULLS FIRST, isnull(b#17L) ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(coalesce(a#16L, 0), isnull(a#16L), coalesce(b#17L, 0), isnull(b#17L), 5), ENSURE_REQUIREMENTS, [id=#67] : +- HashAggregate(keys=[a#16L, b#17L], functions=[]) : +- Exchange hashpartitioning(a#16L, b#17L, 5), ENSURE_REQUIREMENTS, [id=#61] : +- HashAggregate(keys=[a#16L, b#17L], functions=[]) : +- FileScan parquet default.t1[a#16L,b#17L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- Sort [coalesce(c#18L, 0) ASC NULLS FIRST, isnull(c#18L) ASC NULLS FIRST, coalesce(d#19L, 0) ASC NULLS FIRST, isnull(d#19L) ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(coalesce(c#18L, 0), isnull(c#18L), coalesce(d#19L, 0), isnull(d#19L), 5), ENSURE_REQUIREMENTS, [id=#68] +- HashAggregate(keys=[c#18L, d#19L], functions=[]) +- Exchange hashpartitioning(c#18L, d#19L, 5), ENSURE_REQUIREMENTS, [id=#63] +- HashAggregate(keys=[c#18L, d#19L], functions=[]) +- FileScan parquet default.t2[c#18L,d#19L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/yumwang/spark/spark-warehouse/org.apache.spark.sql.Data..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` ### Why are the changes needed? 1. Pushdown LeftSemi/LeftAnti over Aggregate will affect performance. 2. It will remove user added DISTINCT operator, e.g.: [q38](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q38.sql), [q87](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q87.sql). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test and benchmark test. SQL | Before this PR(Seconds) | After this PR(Seconds) -- | -- | -- q14a | 660 | 594 q14b | 660 | 600 q38 | 55 | 29 q87 | 66 | 35 Before this pr: ![image](https://user-images.githubusercontent.com/5399861/104452849-8789fc80-55de-11eb-88da-44059899f9a9.png) After this pr: ![image](https://user-images.githubusercontent.com/5399861/104452899-9a043600-55de-11eb-9286-d8f3a23ca3b8.png) Closes #31145 from wangyum/SPARK-34081. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../optimizer/PushDownLeftSemiAntiJoin.scala | 9 +- .../spark/sql/catalyst/optimizer/joins.scala | 7 + .../LeftSemiAntiJoinPushDownSuite.scala | 25 + .../q14a.sf100/explain.txt | 889 ++++----- .../q14a.sf100/simplified.txt | 225 +-- .../q14b.sf100/explain.txt | 759 ++++---- .../q14b.sf100/simplified.txt | 203 +- .../approved-plans-v1_4/q38.sf100/explain.txt | 458 ++--- .../q38.sf100/simplified.txt | 214 ++- .../approved-plans-v1_4/q87.sf100/explain.txt | 458 ++--- .../q87.sf100/simplified.txt | 214 ++- .../approved-plans-v2_7/q14.sf100/explain.txt | 759 ++++---- .../q14.sf100/simplified.txt | 203 +- .../q14a.sf100/explain.txt | 1705 +++++++++-------- .../q14a.sf100/simplified.txt | 309 +-- 15 files changed, 3316 insertions(+), 3121 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala index 286b447cdb5a8..7eea68663d9ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala @@ -31,7 +31,9 @@ import org.apache.spark.sql.catalyst.rules.Rule * 4) Aggregate * 5) Other permissible unary operators. please see [[PushPredicateThroughNonJoin.canPushThrough]]. */ -object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan] with PredicateHelper { +object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan] + with PredicateHelper + with JoinSelectionHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // LeftSemi/LeftAnti over Project case Join(p @ Project(pList, gChild), rightOp, LeftSemiOrAnti(joinType), joinCond, hint) @@ -51,10 +53,11 @@ object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan] with PredicateHelper { p.copy(child = Join(gChild, rightOp, joinType, newJoinCond, hint)) } - // LeftSemi/LeftAnti over Aggregate + // LeftSemi/LeftAnti over Aggregate, only push down if join can be planned as broadcast join. case join @ Join(agg: Aggregate, rightOp, LeftSemiOrAnti(_), _, _) if agg.aggregateExpressions.forall(_.deterministic) && agg.groupingExpressions.nonEmpty && - !agg.aggregateExpressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) => + !agg.aggregateExpressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) && + canPlanAsBroadcastHashJoin(join, conf) => val aliasMap = getAliasMap(agg) val canPushDownPredicate = (predicate: Expression) => { val replaced = replaceAlias(predicate, aliasMap) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 57c3f3dbd050d..07f3c3f6d8e52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -307,6 +307,13 @@ trait JoinSelectionHelper { } } + def canPlanAsBroadcastHashJoin(join: Join, conf: SQLConf): Boolean = { + getBroadcastBuildSide(join.left, join.right, join.joinType, + join.hint, hintOnly = true, conf).isDefined || + getBroadcastBuildSide(join.left, join.right, join.joinType, + join.hint, hintOnly = false, conf).isDefined + } + def hintToBroadcastLeft(hint: JoinHint): Boolean = { hint.leftHint.exists(_.strategy.contains(BROADCAST)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index d4b85b036b64c..1672c6d91660a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType class LeftSemiPushdownSuite extends PlanTest { @@ -443,4 +444,28 @@ class LeftSemiPushdownSuite extends PlanTest { } } + Seq(LeftSemi, LeftAnti).foreach { jt => + test(s"SPARK-34081: $jt only push down if join can be planned as broadcast join") { + Seq(-1, 100000).foreach { threshold => + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> threshold.toString) { + val originalQuery = testRelation + .groupBy('b)('b) + .join(testRelation1, joinType = jt, condition = Some('b <=> 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = if (threshold > 0) { + testRelation + .join(testRelation1, joinType = jt, condition = Some('b <=> 'd)) + .groupBy('b)('b) + .analyze + } else { + originalQuery.analyze + } + + comparePlans(optimized, correctAnswer) + } + } + } + } + } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index b346701fa3148..11d3efe28f183 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -1,135 +1,138 @@ == Physical Plan == -TakeOrderedAndProject (131) -+- * HashAggregate (130) - +- Exchange (129) - +- * HashAggregate (128) - +- * Expand (127) - +- Union (126) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (134) ++- * HashAggregate (133) + +- Exchange (132) + +- * HashAggregate (131) + +- * Expand (130) + +- Union (129) + :- * Project (90) + : +- * Filter (89) + : +- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- SortMergeJoin LeftSemi (67) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (66) + : : : +- Exchange (65) + : : : +- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - :- * Project (106) - : +- * Filter (105) - : +- * HashAggregate (104) - : +- Exchange (103) - : +- * HashAggregate (102) - : +- * Project (101) - : +- * BroadcastHashJoin Inner BuildRight (100) - : :- * Project (98) - : : +- * BroadcastHashJoin Inner BuildRight (97) - : : :- SortMergeJoin LeftSemi (95) - : : : :- * Sort (92) - : : : : +- Exchange (91) - : : : : +- * Filter (90) - : : : : +- * ColumnarToRow (89) - : : : : +- Scan parquet default.catalog_sales (88) - : : : +- * Sort (94) - : : : +- ReusedExchange (93) - : : +- ReusedExchange (96) - : +- ReusedExchange (99) - +- * Project (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * Project (117) - : +- * BroadcastHashJoin Inner BuildRight (116) - : :- SortMergeJoin LeftSemi (114) - : : :- * Sort (111) - : : : +- Exchange (110) - : : : +- * Filter (109) - : : : +- * ColumnarToRow (108) - : : : +- Scan parquet default.web_sales (107) - : : +- * Sort (113) - : : +- ReusedExchange (112) - : +- ReusedExchange (115) - +- ReusedExchange (118) + : : : +- BroadcastExchange (62) + : : : +- * HashAggregate (61) + : : : +- Exchange (60) + : : : +- * HashAggregate (59) + : : : +- SortMergeJoin LeftSemi (58) + : : : :- * Sort (46) + : : : : +- Exchange (45) + : : : : +- * HashAggregate (44) + : : : : +- Exchange (43) + : : : : +- * HashAggregate (42) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (57) + : : : +- Exchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : :- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.web_sales (47) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (72) + : : +- * Project (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.date_dim (68) + : +- BroadcastExchange (83) + : +- SortMergeJoin LeftSemi (82) + : :- * Sort (79) + : : +- Exchange (78) + : : +- * Filter (77) + : : +- * ColumnarToRow (76) + : : +- Scan parquet default.item (75) + : +- * Sort (81) + : +- ReusedExchange (80) + :- * Project (109) + : +- * Filter (108) + : +- * HashAggregate (107) + : +- Exchange (106) + : +- * HashAggregate (105) + : +- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * Project (101) + : : +- * BroadcastHashJoin Inner BuildRight (100) + : : :- SortMergeJoin LeftSemi (98) + : : : :- * Sort (95) + : : : : +- Exchange (94) + : : : : +- * Filter (93) + : : : : +- * ColumnarToRow (92) + : : : : +- Scan parquet default.catalog_sales (91) + : : : +- * Sort (97) + : : : +- ReusedExchange (96) + : : +- ReusedExchange (99) + : +- ReusedExchange (102) + +- * Project (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * Project (123) + +- * BroadcastHashJoin Inner BuildRight (122) + :- * Project (120) + : +- * BroadcastHashJoin Inner BuildRight (119) + : :- SortMergeJoin LeftSemi (117) + : : :- * Sort (114) + : : : +- Exchange (113) + : : : +- * Filter (112) + : : : +- * ColumnarToRow (111) + : : : +- Scan parquet default.web_sales (110) + : : +- * Sort (116) + : : +- ReusedExchange (115) + : +- ReusedExchange (118) + +- ReusedExchange (121) (1) Scan parquet default.store_sales @@ -148,7 +151,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) (4) Exchange Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#5] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#5] (5) Sort [codegen id : 2] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -161,10 +164,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 17] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(8) Filter [codegen id : 17] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) @@ -242,7 +245,7 @@ Input [5]: [ss_item_sk#2, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id (25) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), true, [id=#17] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#17] (26) Sort [codegen id : 6] Input [3]: [brand_id#14, class_id#15, category_id#16] @@ -303,7 +306,7 @@ Input [5]: [cs_item_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_i (39) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), true, [id=#21] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#21] (40) Sort [codegen id : 10] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] @@ -314,565 +317,577 @@ Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(42) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(43) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#22] + +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(45) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#23] + +(46) Sort [codegen id : 13] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: [coalesce(brand_id#14, 0) ASC NULLS FIRST, isnull(brand_id#14) ASC NULLS FIRST, coalesce(class_id#15, 0) ASC NULLS FIRST, isnull(class_id#15) ASC NULLS FIRST, coalesce(category_id#16, 0) ASC NULLS FIRST, isnull(category_id#16) ASC NULLS FIRST], false, 0 + +(47) Scan parquet default.web_sales +Output [2]: [ws_sold_date_sk#24, ws_item_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] -(44) Filter [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] -Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) +(49) Filter [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] +Condition : (isnotnull(ws_item_sk#25) AND isnotnull(ws_sold_date_sk#24)) -(45) ReusedExchange [Reuses operator id: 16] +(50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#10] -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#22] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(47) Project [codegen id : 13] -Output [1]: [ws_item_sk#23] -Input [3]: [ws_sold_date_sk#22, ws_item_sk#23, d_date_sk#10] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#25] +Input [3]: [ws_sold_date_sk#24, ws_item_sk#25, d_date_sk#10] -(48) ReusedExchange [Reuses operator id: 36] +(53) ReusedExchange [Reuses operator id: 36] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#23] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#6] Join condition: None -(50) Project [codegen id : 13] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_item_sk#23, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_item_sk#25, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(51) Exchange +(56) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), true, [id=#24] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#26] -(52) Sort [codegen id : 14] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [coalesce(i_brand_id#7, 0) ASC NULLS FIRST, isnull(i_brand_id#7) ASC NULLS FIRST, coalesce(i_class_id#8, 0) ASC NULLS FIRST, isnull(i_class_id#8) ASC NULLS FIRST, coalesce(i_category_id#9, 0) ASC NULLS FIRST, isnull(i_category_id#9) ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(58) SortMergeJoin Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16)] Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(54) HashAggregate [codegen id : 15] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(55) Exchange -Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), true, [id=#25] - -(56) HashAggregate [codegen id : 16] +(60) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#27] -(58) HashAggregate [codegen id : 16] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(59) BroadcastExchange +(62) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#28] -(60) BroadcastHashJoin [codegen id : 17] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#6 AS ss_item_sk#29] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange -Input [1]: [ss_item_sk#27] -Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] +(65) Exchange +Input [1]: [ss_item_sk#29] +Arguments: hashpartitioning(ss_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] -(63) Sort [codegen id : 18] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(67) SortMergeJoin Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(65) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#29] +(68) Scan parquet default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] -Input [3]: [d_date_sk#10, d_year#11, d_moy#29] +(69) ColumnarToRow [codegen id : 22] +Input [3]: [d_date_sk#10, d_year#11, d_moy#31] -(67) Filter [codegen id : 19] -Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2001)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) +(70) Filter [codegen id : 22] +Input [3]: [d_date_sk#10, d_year#11, d_moy#31] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#31)) AND (d_year#11 = 2001)) AND (d_moy#31 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#11, d_moy#29] +Input [3]: [d_date_sk#10, d_year#11, d_moy#31] -(69) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] -(70) BroadcastHashJoin [codegen id : 38] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(75) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(78) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#31] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#33] -(76) Sort [codegen id : 21] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(80) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(78) Sort [codegen id : 37] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(82) SortMergeJoin Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(80) BroadcastExchange +(83) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] -(81) BroadcastHashJoin [codegen id : 38] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] +Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -(84) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#39] +(87) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#41] -(85) HashAggregate [codegen id : 39] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] +(88) HashAggregate [codegen id : 45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(86) Filter [codegen id : 39] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(89) Filter [codegen id : 45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(87) Project [codegen id : 39] -Output [6]: [sales#42, number_sales#43, store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +(90) Project [codegen id : 45] +Output [6]: [sales#44, number_sales#45, store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(88) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(91) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(92) ColumnarToRow [codegen id : 46] +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] -(90) Filter [codegen id : 40] -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(93) Filter [codegen id : 46] +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -Arguments: hashpartitioning(cs_item_sk#19, 5), true, [id=#50] +(94) Exchange +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] +Arguments: hashpartitioning(cs_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#52] -(92) Sort [codegen id : 41] -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(95) Sort [codegen id : 47] +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(96) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(94) Sort [codegen id : 57] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 66] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(98) SortMergeJoin Left keys [1]: [cs_item_sk#19] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(99) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(100) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] -Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] -Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] +(101) Project [codegen id : 89] +Output [3]: [cs_item_sk#19, cs_quantity#50, cs_list_price#51] +Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(102) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(103) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] -Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(104) Project [codegen id : 89] +Output [5]: [cs_quantity#50, cs_list_price#51, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [cs_item_sk#19, cs_quantity#50, cs_list_price#51, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] -Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] +(105) HashAggregate [codegen id : 89] +Input [5]: [cs_quantity#50, cs_list_price#51, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#56, isEmpty#57, count#58] -(103) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#57] +(106) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#59] -(104) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] +(107) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#56, isEmpty#57, count#58] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64] -(105) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(108) Filter [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(106) Project [codegen id : 78] -Output [6]: [sales#60, number_sales#61, catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] +(109) Project [codegen id : 90] +Output [6]: [sales#62, number_sales#63, catalog AS channel#65, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64] -(107) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] +(110) Scan parquet default.web_sales +Output [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] +(111) ColumnarToRow [codegen id : 91] +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] -(109) Filter [codegen id : 79] -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) +(112) Filter [codegen id : 91] +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Condition : (isnotnull(ws_item_sk#25) AND isnotnull(ws_sold_date_sk#24)) -(110) Exchange -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Arguments: hashpartitioning(ws_item_sk#23, 5), true, [id=#66] +(113) Exchange +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Arguments: hashpartitioning(ws_item_sk#25, 5), ENSURE_REQUIREMENTS, [id=#68] -(111) Sort [codegen id : 80] -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 92] +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Arguments: [ws_item_sk#25 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(115) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(113) Sort [codegen id : 96] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 111] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +(117) SortMergeJoin +Left keys [1]: [ws_item_sk#25] +Right keys [1]: [ss_item_sk#29] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(118) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] -Left keys [1]: [ws_sold_date_sk#22] +(119) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] -Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] +(120) Project [codegen id : 134] +Output [3]: [ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Input [5]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(121) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] -Left keys [1]: [ws_item_sk#23] +(122) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] -Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(123) Project [codegen id : 134] +Output [5]: [ws_quantity#66, ws_list_price#67, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ws_item_sk#25, ws_quantity#66, ws_list_price#67, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] -Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] +(124) HashAggregate [codegen id : 134] +Input [5]: [ws_quantity#66, ws_list_price#67, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#72, isEmpty#73, count#74] -(122) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#73] +(125) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#75] -(123) HashAggregate [codegen id : 117] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] +(126) HashAggregate [codegen id : 135] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#72, isEmpty#73, count#74] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] - -(124) Filter [codegen id : 117] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) - -(125) Project [codegen id : 117] -Output [6]: [sales#76, number_sales#77, web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] - -(126) Union - -(127) Expand [codegen id : 118] -Input [6]: [sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, null, 1), List(sales#42, number_sales#43, channel#47, i_brand_id#7, null, null, 3), List(sales#42, number_sales#43, channel#47, null, null, null, 7), List(sales#42, number_sales#43, null, null, null, null, 15)], [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] - -(128) HashAggregate [codegen id : 118] -Input [7]: [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] -Aggregate Attributes [3]: [sum#85, isEmpty#86, sum#87] -Results [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] - -(129) Exchange -Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, 5), true, [id=#91] - -(130) HashAggregate [codegen id : 119] -Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] -Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -Functions [2]: [sum(sales#42), sum(number_sales#43)] -Aggregate Attributes [2]: [sum(sales#42)#92, sum(number_sales#43)#93] -Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#42)#92 AS sum(sales)#94, sum(number_sales#43)#93 AS sum(number_sales)#95] - -(131) TakeOrderedAndProject -Input [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] -Arguments: 100, [channel#80 ASC NULLS FIRST, i_brand_id#81 ASC NULLS FIRST, i_class_id#82 ASC NULLS FIRST, i_category_id#83 ASC NULLS FIRST], [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80] + +(127) Filter [codegen id : 135] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) + +(128) Project [codegen id : 135] +Output [6]: [sales#78, number_sales#79, web AS channel#81, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80] + +(129) Union + +(130) Expand [codegen id : 136] +Input [6]: [sales#44, number_sales#45, channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: [List(sales#44, number_sales#45, channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#44, number_sales#45, channel#49, i_brand_id#7, i_class_id#8, null, 1), List(sales#44, number_sales#45, channel#49, i_brand_id#7, null, null, 3), List(sales#44, number_sales#45, channel#49, null, null, null, 7), List(sales#44, number_sales#45, null, null, null, null, 15)], [sales#44, number_sales#45, channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86] + +(131) HashAggregate [codegen id : 136] +Input [7]: [sales#44, number_sales#45, channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86] +Keys [5]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86] +Functions [2]: [partial_sum(sales#44), partial_sum(number_sales#45)] +Aggregate Attributes [3]: [sum#87, isEmpty#88, sum#89] +Results [8]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86, sum#90, isEmpty#91, sum#92] + +(132) Exchange +Input [8]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86, sum#90, isEmpty#91, sum#92] +Arguments: hashpartitioning(channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86, 5), ENSURE_REQUIREMENTS, [id=#93] + +(133) HashAggregate [codegen id : 137] +Input [8]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86, sum#90, isEmpty#91, sum#92] +Keys [5]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, spark_grouping_id#86] +Functions [2]: [sum(sales#44), sum(number_sales#45)] +Aggregate Attributes [2]: [sum(sales#44)#94, sum(number_sales#45)#95] +Results [6]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, sum(sales#44)#94 AS sum(sales)#96, sum(number_sales#45)#95 AS sum(number_sales)#97] + +(134) TakeOrderedAndProject +Input [6]: [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, sum(sales)#96, sum(number_sales)#97] +Arguments: 100, [channel#82 ASC NULLS FIRST, i_brand_id#83 ASC NULLS FIRST, i_class_id#84 ASC NULLS FIRST, i_category_id#85 ASC NULLS FIRST], [channel#82, i_brand_id#83, i_class_id#84, i_category_id#85, sum(sales)#96, sum(number_sales)#97] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (157) -+- Exchange (156) - +- * HashAggregate (155) - +- Union (154) - :- * Project (141) - : +- * BroadcastHashJoin Inner BuildRight (140) - : :- * Filter (134) - : : +- * ColumnarToRow (133) - : : +- Scan parquet default.store_sales (132) - : +- BroadcastExchange (139) - : +- * Project (138) - : +- * Filter (137) - : +- * ColumnarToRow (136) - : +- Scan parquet default.date_dim (135) - :- * Project (147) - : +- * BroadcastHashJoin Inner BuildRight (146) - : :- * Filter (144) - : : +- * ColumnarToRow (143) - : : +- Scan parquet default.catalog_sales (142) - : +- ReusedExchange (145) - +- * Project (153) - +- * BroadcastHashJoin Inner BuildRight (152) - :- * Filter (150) - : +- * ColumnarToRow (149) - : +- Scan parquet default.web_sales (148) - +- ReusedExchange (151) - - -(132) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (160) ++- Exchange (159) + +- * HashAggregate (158) + +- Union (157) + :- * Project (144) + : +- * BroadcastHashJoin Inner BuildRight (143) + : :- * Filter (137) + : : +- * ColumnarToRow (136) + : : +- Scan parquet default.store_sales (135) + : +- BroadcastExchange (142) + : +- * Project (141) + : +- * Filter (140) + : +- * ColumnarToRow (139) + : +- Scan parquet default.date_dim (138) + :- * Project (150) + : +- * BroadcastHashJoin Inner BuildRight (149) + : :- * Filter (147) + : : +- * ColumnarToRow (146) + : : +- Scan parquet default.catalog_sales (145) + : +- ReusedExchange (148) + +- * Project (156) + +- * BroadcastHashJoin Inner BuildRight (155) + :- * Filter (153) + : +- * ColumnarToRow (152) + : +- Scan parquet default.web_sales (151) + +- ReusedExchange (154) + + +(135) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] +(136) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(134) Filter [codegen id : 2] +(137) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(135) Scan parquet default.date_dim +(138) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(136) ColumnarToRow [codegen id : 1] +(139) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(137) Filter [codegen id : 1] +(140) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(138) Project [codegen id : 1] +(141) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(139) BroadcastExchange +(142) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] -(140) BroadcastHashJoin [codegen id : 2] +(143) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(141) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#97, ss_list_price#4 AS list_price#98] +(144) Project [codegen id : 2] +Output [2]: [ss_quantity#3 AS quantity#99, ss_list_price#4 AS list_price#100] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(142) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] +(145) Scan parquet default.catalog_sales +Output [3]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(143) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] +(146) ColumnarToRow [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51] -(144) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] +(147) Filter [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51] Condition : isnotnull(cs_sold_date_sk#18) -(145) ReusedExchange [Reuses operator id: 139] +(148) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#10] -(146) BroadcastHashJoin [codegen id : 4] +(149) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(147) Project [codegen id : 4] -Output [2]: [cs_quantity#48 AS quantity#99, cs_list_price#49 AS list_price#100] -Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] +(150) Project [codegen id : 4] +Output [2]: [cs_quantity#50 AS quantity#101, cs_list_price#51 AS list_price#102] +Input [4]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51, d_date_sk#10] -(148) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] +(151) Scan parquet default.web_sales +Output [3]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(149) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] +(152) ColumnarToRow [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67] -(150) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -Condition : isnotnull(ws_sold_date_sk#22) +(153) Filter [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67] +Condition : isnotnull(ws_sold_date_sk#24) -(151) ReusedExchange [Reuses operator id: 139] +(154) ReusedExchange [Reuses operator id: 142] Output [1]: [d_date_sk#10] -(152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#22] +(155) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(153) Project [codegen id : 6] -Output [2]: [ws_quantity#64 AS quantity#101, ws_list_price#65 AS list_price#102] -Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] +(156) Project [codegen id : 6] +Output [2]: [ws_quantity#66 AS quantity#103, ws_list_price#67 AS list_price#104] +Input [4]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67, d_date_sk#10] -(154) Union +(157) Union -(155) HashAggregate [codegen id : 7] -Input [2]: [quantity#97, list_price#98] +(158) HashAggregate [codegen id : 7] +Input [2]: [quantity#99, list_price#100] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#105, count#106] +Results [2]: [sum#107, count#108] -(156) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, true, [id=#107] +(159) Exchange +Input [2]: [sum#107, count#108] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#109] -(157) HashAggregate [codegen id : 8] -Input [2]: [sum#105, count#106] +(160) HashAggregate [codegen id : 8] +Input [2]: [sum#107, count#108] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108 AS average_sales#109] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#110] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#110 AS average_sales#111] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 5b93392d023db..dd90e20fb61e2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (119) + WholeStageCodegen (137) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (118) + WholeStageCodegen (136) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (39) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter - Exchange #16 + Exchange #18 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -28,7 +28,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter - BroadcastExchange #17 + BroadcastExchange #19 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -43,7 +43,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter - ReusedExchange [d_date_sk] #17 + ReusedExchange [d_date_sk] #19 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -52,11 +52,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter - ReusedExchange [d_date_sk] #17 + ReusedExchange [d_date_sk] #19 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (38) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -73,11 +73,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - WholeStageCodegen (18) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (17) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -86,116 +86,125 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (16) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #8 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #10 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) + BroadcastExchange #15 + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #14 + BroadcastExchange #16 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (21) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #15 - WholeStageCodegen (20) + Exchange [i_item_sk] #17 + WholeStageCodegen (23) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (37) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 - WholeStageCodegen (78) + WholeStageCodegen (90) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #18 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #20 + WholeStageCodegen (89) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -203,31 +212,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (41) + WholeStageCodegen (47) Sort [cs_item_sk] InputAdapter - Exchange [cs_item_sk] #19 - WholeStageCodegen (40) + Exchange [cs_item_sk] #21 + WholeStageCodegen (46) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] - WholeStageCodegen (57) + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #13 + ReusedExchange [d_date_sk] #15 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (117) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (135) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #20 - WholeStageCodegen (116) + Exchange [i_brand_id,i_class_id,i_category_id] #22 + WholeStageCodegen (134) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -235,20 +244,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (80) + WholeStageCodegen (92) Sort [ws_item_sk] InputAdapter - Exchange [ws_item_sk] #21 - WholeStageCodegen (79) + Exchange [ws_item_sk] #23 + WholeStageCodegen (91) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] - WholeStageCodegen (96) + WholeStageCodegen (111) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #13 + ReusedExchange [d_date_sk] #15 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 2d2b56e32bdb8..417d2895c8630 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -1,114 +1,117 @@ == Physical Plan == -TakeOrderedAndProject (110) -+- * BroadcastHashJoin Inner BuildRight (109) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (113) ++- * BroadcastHashJoin Inner BuildRight (112) + :- * Project (90) + : +- * Filter (89) + : +- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- SortMergeJoin LeftSemi (67) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (66) + : : : +- Exchange (65) + : : : +- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - +- BroadcastExchange (108) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- SortMergeJoin LeftSemi (92) - : : :- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (91) - : : +- ReusedExchange (90) - : +- BroadcastExchange (97) - : +- * Project (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet default.date_dim (93) - +- ReusedExchange (100) + : : : +- BroadcastExchange (62) + : : : +- * HashAggregate (61) + : : : +- Exchange (60) + : : : +- * HashAggregate (59) + : : : +- SortMergeJoin LeftSemi (58) + : : : :- * Sort (46) + : : : : +- Exchange (45) + : : : : +- * HashAggregate (44) + : : : : +- Exchange (43) + : : : : +- * HashAggregate (42) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (57) + : : : +- Exchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : :- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.web_sales (47) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (72) + : : +- * Project (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.date_dim (68) + : +- BroadcastExchange (83) + : +- SortMergeJoin LeftSemi (82) + : :- * Sort (79) + : : +- Exchange (78) + : : +- * Filter (77) + : : +- * ColumnarToRow (76) + : : +- Scan parquet default.item (75) + : +- * Sort (81) + : +- ReusedExchange (80) + +- BroadcastExchange (111) + +- * Project (110) + +- * Filter (109) + +- * HashAggregate (108) + +- Exchange (107) + +- * HashAggregate (106) + +- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (102) + : +- * BroadcastHashJoin Inner BuildRight (101) + : :- SortMergeJoin LeftSemi (95) + : : :- * Sort (92) + : : : +- ReusedExchange (91) + : : +- * Sort (94) + : : +- ReusedExchange (93) + : +- BroadcastExchange (100) + : +- * Project (99) + : +- * Filter (98) + : +- * ColumnarToRow (97) + : +- Scan parquet default.date_dim (96) + +- ReusedExchange (103) (1) Scan parquet default.store_sales @@ -127,7 +130,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) (4) Exchange Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#5] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#5] (5) Sort [codegen id : 2] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -140,10 +143,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 17] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(8) Filter [codegen id : 17] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) @@ -221,7 +224,7 @@ Input [5]: [ss_item_sk#2, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id (25) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), true, [id=#17] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#17] (26) Sort [codegen id : 6] Input [3]: [brand_id#14, class_id#15, category_id#16] @@ -282,7 +285,7 @@ Input [5]: [cs_item_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_i (39) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), true, [id=#21] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#21] (40) Sort [codegen id : 10] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] @@ -293,518 +296,530 @@ Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(42) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(43) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#22] + +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(45) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#23] + +(46) Sort [codegen id : 13] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: [coalesce(brand_id#14, 0) ASC NULLS FIRST, isnull(brand_id#14) ASC NULLS FIRST, coalesce(class_id#15, 0) ASC NULLS FIRST, isnull(class_id#15) ASC NULLS FIRST, coalesce(category_id#16, 0) ASC NULLS FIRST, isnull(category_id#16) ASC NULLS FIRST], false, 0 + +(47) Scan parquet default.web_sales +Output [2]: [ws_sold_date_sk#24, ws_item_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] -(44) Filter [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] -Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) +(49) Filter [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] +Condition : (isnotnull(ws_item_sk#25) AND isnotnull(ws_sold_date_sk#24)) -(45) ReusedExchange [Reuses operator id: 16] +(50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#10] -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#22] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(47) Project [codegen id : 13] -Output [1]: [ws_item_sk#23] -Input [3]: [ws_sold_date_sk#22, ws_item_sk#23, d_date_sk#10] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#25] +Input [3]: [ws_sold_date_sk#24, ws_item_sk#25, d_date_sk#10] -(48) ReusedExchange [Reuses operator id: 36] +(53) ReusedExchange [Reuses operator id: 36] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#23] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#6] Join condition: None -(50) Project [codegen id : 13] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_item_sk#23, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_item_sk#25, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(51) Exchange +(56) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), true, [id=#24] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#26] -(52) Sort [codegen id : 14] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [coalesce(i_brand_id#7, 0) ASC NULLS FIRST, isnull(i_brand_id#7) ASC NULLS FIRST, coalesce(i_class_id#8, 0) ASC NULLS FIRST, isnull(i_class_id#8) ASC NULLS FIRST, coalesce(i_category_id#9, 0) ASC NULLS FIRST, isnull(i_category_id#9) ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(58) SortMergeJoin Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16)] Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(54) HashAggregate [codegen id : 15] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(55) Exchange -Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), true, [id=#25] - -(56) HashAggregate [codegen id : 16] +(60) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#27] -(58) HashAggregate [codegen id : 16] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(59) BroadcastExchange +(62) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#28] -(60) BroadcastHashJoin [codegen id : 17] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#6 AS ss_item_sk#29] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange -Input [1]: [ss_item_sk#27] -Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] +(65) Exchange +Input [1]: [ss_item_sk#29] +Arguments: hashpartitioning(ss_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] -(63) Sort [codegen id : 18] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(67) SortMergeJoin Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(65) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_week_seq#29] +(68) Scan parquet default.date_dim +Output [2]: [d_date_sk#10, d_week_seq#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] -Input [2]: [d_date_sk#10, d_week_seq#29] +(69) ColumnarToRow [codegen id : 22] +Input [2]: [d_date_sk#10, d_week_seq#31] -(67) Filter [codegen id : 19] -Input [2]: [d_date_sk#10, d_week_seq#29] -Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#30, [id=#31])) AND isnotnull(d_date_sk#10)) +(70) Filter [codegen id : 22] +Input [2]: [d_date_sk#10, d_week_seq#31] +Condition : ((isnotnull(d_week_seq#31) AND (d_week_seq#31 = Subquery scalar-subquery#32, [id=#33])) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_week_seq#29] +Input [2]: [d_date_sk#10, d_week_seq#31] -(69) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(70) BroadcastHashJoin [codegen id : 38] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(75) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(75) Exchange +(78) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#33] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#35] -(76) Sort [codegen id : 21] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(80) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(78) Sort [codegen id : 37] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(82) SortMergeJoin Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(80) BroadcastExchange +(83) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] -(81) BroadcastHashJoin [codegen id : 38] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] +Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#40, isEmpty#41, count#42] -(84) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#41] +(87) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#40, isEmpty#41, count#42] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#43] -(85) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] +(88) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#40, isEmpty#41, count#42] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48] -(86) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) +(89) Filter [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(87) Project [codegen id : 78] -Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +(90) Project [codegen id : 90] +Output [6]: [store AS channel#51, i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48] -(88) ReusedExchange [Reuses operator id: 4] +(91) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(89) Sort [codegen id : 40] +(92) Sort [codegen id : 46] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(93) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(91) Sort [codegen id : 56] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(94) Sort [codegen id : 65] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(95) SortMergeJoin Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(93) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_week_seq#29] +(96) Scan parquet default.date_dim +Output [2]: [d_date_sk#10, d_week_seq#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 57] -Input [2]: [d_date_sk#10, d_week_seq#29] +(97) ColumnarToRow [codegen id : 66] +Input [2]: [d_date_sk#10, d_week_seq#31] -(95) Filter [codegen id : 57] -Input [2]: [d_date_sk#10, d_week_seq#29] -Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#10)) +(98) Filter [codegen id : 66] +Input [2]: [d_date_sk#10, d_week_seq#31] +Condition : ((isnotnull(d_week_seq#31) AND (d_week_seq#31 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 57] +(99) Project [codegen id : 66] Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_week_seq#29] +Input [2]: [d_date_sk#10, d_week_seq#31] -(97) BroadcastExchange +(100) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] -(98) BroadcastHashJoin [codegen id : 76] +(101) BroadcastHashJoin [codegen id : 88] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(99) Project [codegen id : 76] +(102) Project [codegen id : 88] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(100) ReusedExchange [Reuses operator id: 80] -Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(103) ReusedExchange [Reuses operator id: 83] +Output [4]: [i_item_sk#55, i_brand_id#56, i_class_id#57, i_category_id#58] -(101) BroadcastHashJoin [codegen id : 76] +(104) BroadcastHashJoin [codegen id : 88] Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#53] +Right keys [1]: [i_item_sk#55] Join condition: None -(102) Project [codegen id : 76] -Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(105) Project [codegen id : 88] +Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#56, i_class_id#57, i_category_id#58] +Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#55, i_brand_id#56, i_class_id#57, i_category_id#58] -(103) HashAggregate [codegen id : 76] -Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(106) HashAggregate [codegen id : 88] +Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#56, i_class_id#57, i_category_id#58] +Keys [3]: [i_brand_id#56, i_class_id#57, i_category_id#58] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum#62, isEmpty#63, count#64] -(104) Exchange -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), true, [id=#63] +(107) Exchange +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#56, i_class_id#57, i_category_id#58, 5), ENSURE_REQUIREMENTS, [id=#65] -(105) HashAggregate [codegen id : 77] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(108) HashAggregate [codegen id : 89] +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#56, i_class_id#57, i_category_id#58] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70] -(106) Filter [codegen id : 77] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) +(109) Filter [codegen id : 89] +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(107) Project [codegen id : 77] -Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] +(110) Project [codegen id : 89] +Output [6]: [store AS channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70] -(108) BroadcastExchange -Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +(111) BroadcastExchange +Input [6]: [channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#72] -(109) BroadcastHashJoin [codegen id : 78] +(112) BroadcastHashJoin [codegen id : 90] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +Right keys [3]: [i_brand_id#56, i_class_id#57, i_category_id#58] Join condition: None -(110) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] +(113) TakeOrderedAndProject +Input [12]: [channel#51, i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#51, i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (136) -+- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.store_sales (111) - : +- BroadcastExchange (118) - : +- * Project (117) - : +- * Filter (116) - : +- * ColumnarToRow (115) - : +- Scan parquet default.date_dim (114) - :- * Project (126) - : +- * BroadcastHashJoin Inner BuildRight (125) - : :- * Filter (123) - : : +- * ColumnarToRow (122) - : : +- Scan parquet default.catalog_sales (121) - : +- ReusedExchange (124) - +- * Project (132) - +- * BroadcastHashJoin Inner BuildRight (131) - :- * Filter (129) - : +- * ColumnarToRow (128) - : +- Scan parquet default.web_sales (127) - +- ReusedExchange (130) - - -(111) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (139) ++- Exchange (138) + +- * HashAggregate (137) + +- Union (136) + :- * Project (123) + : +- * BroadcastHashJoin Inner BuildRight (122) + : :- * Filter (116) + : : +- * ColumnarToRow (115) + : : +- Scan parquet default.store_sales (114) + : +- BroadcastExchange (121) + : +- * Project (120) + : +- * Filter (119) + : +- * ColumnarToRow (118) + : +- Scan parquet default.date_dim (117) + :- * Project (129) + : +- * BroadcastHashJoin Inner BuildRight (128) + : :- * Filter (126) + : : +- * ColumnarToRow (125) + : : +- Scan parquet default.catalog_sales (124) + : +- ReusedExchange (127) + +- * Project (135) + +- * BroadcastHashJoin Inner BuildRight (134) + :- * Filter (132) + : +- * ColumnarToRow (131) + : +- Scan parquet default.web_sales (130) + +- ReusedExchange (133) + + +(114) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(115) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(113) Filter [codegen id : 2] +(116) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(114) Scan parquet default.date_dim +(117) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(115) ColumnarToRow [codegen id : 1] +(118) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(116) Filter [codegen id : 1] +(119) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(117) Project [codegen id : 1] +(120) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(118) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] -(119) BroadcastHashJoin [codegen id : 2] +(122) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(120) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#72, ss_list_price#4 AS list_price#73] +(123) Project [codegen id : 2] +Output [2]: [ss_quantity#3 AS quantity#74, ss_list_price#4 AS list_price#75] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(121) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] +(124) Scan parquet default.catalog_sales +Output [3]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] +(125) ColumnarToRow [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77] -(123) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] +(126) Filter [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77] Condition : isnotnull(cs_sold_date_sk#18) -(124) ReusedExchange [Reuses operator id: 118] +(127) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#10] -(125) BroadcastHashJoin [codegen id : 4] +(128) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(126) Project [codegen id : 4] -Output [2]: [cs_quantity#74 AS quantity#76, cs_list_price#75 AS list_price#77] -Input [4]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75, d_date_sk#10] +(129) Project [codegen id : 4] +Output [2]: [cs_quantity#76 AS quantity#78, cs_list_price#77 AS list_price#79] +Input [4]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77, d_date_sk#10] -(127) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] +(130) Scan parquet default.web_sales +Output [3]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] +(131) ColumnarToRow [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81] -(129) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] -Condition : isnotnull(ws_sold_date_sk#22) +(132) Filter [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81] +Condition : isnotnull(ws_sold_date_sk#24) -(130) ReusedExchange [Reuses operator id: 118] +(133) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#10] -(131) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#22] +(134) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(132) Project [codegen id : 6] -Output [2]: [ws_quantity#78 AS quantity#80, ws_list_price#79 AS list_price#81] -Input [4]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79, d_date_sk#10] +(135) Project [codegen id : 6] +Output [2]: [ws_quantity#80 AS quantity#82, ws_list_price#81 AS list_price#83] +Input [4]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81, d_date_sk#10] -(133) Union +(136) Union -(134) HashAggregate [codegen id : 7] -Input [2]: [quantity#72, list_price#73] +(137) HashAggregate [codegen id : 7] +Input [2]: [quantity#74, list_price#75] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#82, count#83] -Results [2]: [sum#84, count#85] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] -(135) Exchange -Input [2]: [sum#84, count#85] -Arguments: SinglePartition, true, [id=#86] +(138) Exchange +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] -(136) HashAggregate [codegen id : 8] -Input [2]: [sum#84, count#85] +(139) HashAggregate [codegen id : 8] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87 AS average_sales#88] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#89] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#89 AS average_sales#90] -Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* Project (140) -+- * Filter (139) - +- * ColumnarToRow (138) - +- Scan parquet default.date_dim (137) +Subquery:2 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#32, [id=#33] +* Project (143) ++- * Filter (142) + +- * ColumnarToRow (141) + +- Scan parquet default.date_dim (140) -(137) Scan parquet default.date_dim -Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(140) Scan parquet default.date_dim +Output [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(141) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] -(139) Filter [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 2000)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11)) +(142) Filter [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#91)) AND isnotnull(d_dom#92)) AND (d_year#11 = 2000)) AND (d_moy#91 = 12)) AND (d_dom#92 = 11)) -(140) Project [codegen id : 1] -Output [1]: [d_week_seq#29] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(143) Project [codegen id : 1] +Output [1]: [d_week_seq#31] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] +Subquery:3 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:4 Hosting operator id = 98 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* Project (147) ++- * Filter (146) + +- * ColumnarToRow (145) + +- Scan parquet default.date_dim (144) -(141) Scan parquet default.date_dim -Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(144) Scan parquet default.date_dim +Output [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(145) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] -(143) Filter [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11)) +(146) Filter [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#91)) AND isnotnull(d_dom#92)) AND (d_year#11 = 1999)) AND (d_moy#91 = 12)) AND (d_dom#92 = 11)) -(144) Project [codegen id : 1] -Output [1]: [d_week_seq#29] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(147) Project [codegen id : 1] +Output [1]: [d_week_seq#31] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index d6b8ba4395d2e..f6ebac11d2742 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (78) + WholeStageCodegen (90) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -7,7 +7,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter - Exchange #15 + Exchange #17 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter - BroadcastExchange #16 + BroadcastExchange #18 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -35,7 +35,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter - ReusedExchange [d_date_sk] #16 + ReusedExchange [d_date_sk] #18 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -44,11 +44,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter - ReusedExchange [d_date_sk] #16 + ReusedExchange [d_date_sk] #18 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (38) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -65,11 +65,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - WholeStageCodegen (18) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #3 - WholeStageCodegen (17) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -78,87 +78,96 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (16) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #9 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (19) + BroadcastExchange #14 + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #1 @@ -172,31 +181,31 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #13 + BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (21) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #14 - WholeStageCodegen (20) + Exchange [i_item_sk] #16 + WholeStageCodegen (23) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (37) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (77) + BroadcastExchange #19 + WholeStageCodegen (89) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #18 - WholeStageCodegen (76) + Exchange [i_brand_id,i_class_id,i_category_id] #20 + WholeStageCodegen (88) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -204,17 +213,17 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (40) + WholeStageCodegen (46) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] #2 - WholeStageCodegen (56) + WholeStageCodegen (65) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - BroadcastExchange #19 - WholeStageCodegen (57) + BroadcastExchange #21 + WholeStageCodegen (66) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -228,4 +237,4 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 7465ddae84e8a..00327ece7607a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftSemi (57) - :- SortMergeJoin LeftSemi (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (71) ++- Exchange (70) + +- * HashAggregate (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- SortMergeJoin LeftSemi (65) + :- * Sort (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- Exchange (44) + : +- * HashAggregate (43) + : +- SortMergeJoin LeftSemi (42) + : :- * Sort (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * SortMergeJoin Inner (35) + : :- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Filter (27) + : : : +- * ColumnarToRow (26) + : : : +- Scan parquet default.catalog_sales (25) + : : +- ReusedExchange (28) + : +- * Sort (34) + : +- ReusedExchange (33) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * SortMergeJoin Inner (58) + :- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Filter (50) + : : +- * ColumnarToRow (49) + : : +- Scan parquet default.web_sales (48) + : +- ReusedExchange (51) + +- * Sort (57) + +- ReusedExchange (56) (1) Scan parquet default.store_sales @@ -115,7 +119,7 @@ Input [4]: [ss_sold_date_sk#1, ss_customer_sk#2, d_date_sk#3, d_date#4] (11) Exchange Input [2]: [ss_customer_sk#2, d_date#4] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#7] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] (12) Sort [codegen id : 3] Input [2]: [ss_customer_sk#2, d_date#4] @@ -137,7 +141,7 @@ Condition : isnotnull(c_customer_sk#8) (16) Exchange Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(c_customer_sk#8, 5), true, [id=#11] +Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#11] (17) Sort [codegen id : 5] Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] @@ -149,240 +153,256 @@ Right keys [1]: [c_customer_sk#8] Join condition: None (19) Project [codegen id : 6] -Output [3]: [d_date#4, c_first_name#9, c_last_name#10] +Output [3]: [c_last_name#10, c_first_name#9, d_date#4] Input [5]: [ss_customer_sk#2, d_date#4, c_customer_sk#8, c_first_name#9, c_last_name#10] -(20) Exchange -Input [3]: [d_date#4, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), true, [id=#12] +(20) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(21) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#12] + +(22) HashAggregate [codegen id : 7] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(23) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), ENSURE_REQUIREMENTS, [id=#13] -(21) Sort [codegen id : 7] -Input [3]: [d_date#4, c_first_name#9, c_last_name#10] +(24) Sort [codegen id : 8] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Arguments: [coalesce(c_last_name#10, ) ASC NULLS FIRST, isnull(c_last_name#10) ASC NULLS FIRST, coalesce(c_first_name#9, ) ASC NULLS FIRST, isnull(c_first_name#9) ASC NULLS FIRST, coalesce(d_date#4, 0) ASC NULLS FIRST, isnull(d_date#4) ASC NULLS FIRST], false, 0 -(22) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#13, cs_bill_customer_sk#14] +(25) Scan parquet default.catalog_sales +Output [2]: [cs_sold_date_sk#14, cs_bill_customer_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 9] -Input [2]: [cs_sold_date_sk#13, cs_bill_customer_sk#14] +(26) ColumnarToRow [codegen id : 10] +Input [2]: [cs_sold_date_sk#14, cs_bill_customer_sk#15] -(24) Filter [codegen id : 9] -Input [2]: [cs_sold_date_sk#13, cs_bill_customer_sk#14] -Condition : (isnotnull(cs_sold_date_sk#13) AND isnotnull(cs_bill_customer_sk#14)) +(27) Filter [codegen id : 10] +Input [2]: [cs_sold_date_sk#14, cs_bill_customer_sk#15] +Condition : (isnotnull(cs_sold_date_sk#14) AND isnotnull(cs_bill_customer_sk#15)) -(25) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#15, d_date#16] +(28) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#16, d_date#17] -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] +(29) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] Join condition: None -(27) Project [codegen id : 9] -Output [2]: [cs_bill_customer_sk#14, d_date#16] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, d_date_sk#15, d_date#16] +(30) Project [codegen id : 10] +Output [2]: [cs_bill_customer_sk#15, d_date#17] +Input [4]: [cs_sold_date_sk#14, cs_bill_customer_sk#15, d_date_sk#16, d_date#17] -(28) Exchange -Input [2]: [cs_bill_customer_sk#14, d_date#16] -Arguments: hashpartitioning(cs_bill_customer_sk#14, 5), true, [id=#17] +(31) Exchange +Input [2]: [cs_bill_customer_sk#15, d_date#17] +Arguments: hashpartitioning(cs_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] -(29) Sort [codegen id : 10] -Input [2]: [cs_bill_customer_sk#14, d_date#16] -Arguments: [cs_bill_customer_sk#14 ASC NULLS FIRST], false, 0 +(32) Sort [codegen id : 11] +Input [2]: [cs_bill_customer_sk#15, d_date#17] +Arguments: [cs_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(30) ReusedExchange [Reuses operator id: 16] -Output [3]: [c_customer_sk#18, c_first_name#19, c_last_name#20] +(33) ReusedExchange [Reuses operator id: 16] +Output [3]: [c_customer_sk#19, c_first_name#20, c_last_name#21] -(31) Sort [codegen id : 12] -Input [3]: [c_customer_sk#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 +(34) Sort [codegen id : 13] +Input [3]: [c_customer_sk#19, c_first_name#20, c_last_name#21] +Arguments: [c_customer_sk#19 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#14] -Right keys [1]: [c_customer_sk#18] +(35) SortMergeJoin [codegen id : 14] +Left keys [1]: [cs_bill_customer_sk#15] +Right keys [1]: [c_customer_sk#19] Join condition: None -(33) Project [codegen id : 13] -Output [3]: [c_last_name#20, c_first_name#19, d_date#16] -Input [5]: [cs_bill_customer_sk#14, d_date#16, c_customer_sk#18, c_first_name#19, c_last_name#20] +(36) Project [codegen id : 14] +Output [3]: [c_last_name#21, c_first_name#20, d_date#17] +Input [5]: [cs_bill_customer_sk#15, d_date#17, c_customer_sk#19, c_first_name#20, c_last_name#21] -(34) HashAggregate [codegen id : 13] -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Keys [3]: [c_last_name#20, c_first_name#19, d_date#16] +(37) HashAggregate [codegen id : 14] +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Keys [3]: [c_last_name#21, c_first_name#20, d_date#17] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#20, c_first_name#19, d_date#16] +Results [3]: [c_last_name#21, c_first_name#20, d_date#17] -(35) Exchange -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(c_last_name#20, c_first_name#19, d_date#16, 5), true, [id=#21] +(38) Exchange +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Arguments: hashpartitioning(c_last_name#21, c_first_name#20, d_date#17, 5), ENSURE_REQUIREMENTS, [id=#22] -(36) HashAggregate [codegen id : 14] -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Keys [3]: [c_last_name#20, c_first_name#19, d_date#16] +(39) HashAggregate [codegen id : 15] +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Keys [3]: [c_last_name#21, c_first_name#20, d_date#17] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#20, c_first_name#19, d_date#16] +Results [3]: [c_last_name#21, c_first_name#20, d_date#17] -(37) Exchange -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16), 5), true, [id=#22] +(40) Exchange +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Arguments: hashpartitioning(coalesce(c_last_name#21, ), isnull(c_last_name#21), coalesce(c_first_name#20, ), isnull(c_first_name#20), coalesce(d_date#17, 0), isnull(d_date#17), 5), ENSURE_REQUIREMENTS, [id=#23] -(38) Sort [codegen id : 15] -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: [coalesce(c_last_name#20, ) ASC NULLS FIRST, isnull(c_last_name#20) ASC NULLS FIRST, coalesce(c_first_name#19, ) ASC NULLS FIRST, isnull(c_first_name#19) ASC NULLS FIRST, coalesce(d_date#16, 0) ASC NULLS FIRST, isnull(d_date#16) ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 16] +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Arguments: [coalesce(c_last_name#21, ) ASC NULLS FIRST, isnull(c_last_name#21) ASC NULLS FIRST, coalesce(c_first_name#20, ) ASC NULLS FIRST, isnull(c_first_name#20) ASC NULLS FIRST, coalesce(d_date#17, 0) ASC NULLS FIRST, isnull(d_date#17) ASC NULLS FIRST], false, 0 -(39) SortMergeJoin +(42) SortMergeJoin Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16)] +Right keys [6]: [coalesce(c_last_name#21, ), isnull(c_last_name#21), coalesce(c_first_name#20, ), isnull(c_first_name#20), coalesce(d_date#17, 0), isnull(d_date#17)] Join condition: None -(40) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#23, ws_bill_customer_sk#24] +(43) HashAggregate [codegen id : 17] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(44) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#24] + +(45) HashAggregate [codegen id : 18] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(46) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), ENSURE_REQUIREMENTS, [id=#25] + +(47) Sort [codegen id : 19] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: [coalesce(c_last_name#10, ) ASC NULLS FIRST, isnull(c_last_name#10) ASC NULLS FIRST, coalesce(c_first_name#9, ) ASC NULLS FIRST, isnull(c_first_name#9) ASC NULLS FIRST, coalesce(d_date#4, 0) ASC NULLS FIRST, isnull(d_date#4) ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.web_sales +Output [2]: [ws_sold_date_sk#26, ws_bill_customer_sk#27] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 17] -Input [2]: [ws_sold_date_sk#23, ws_bill_customer_sk#24] +(49) ColumnarToRow [codegen id : 21] +Input [2]: [ws_sold_date_sk#26, ws_bill_customer_sk#27] -(42) Filter [codegen id : 17] -Input [2]: [ws_sold_date_sk#23, ws_bill_customer_sk#24] -Condition : (isnotnull(ws_sold_date_sk#23) AND isnotnull(ws_bill_customer_sk#24)) +(50) Filter [codegen id : 21] +Input [2]: [ws_sold_date_sk#26, ws_bill_customer_sk#27] +Condition : (isnotnull(ws_sold_date_sk#26) AND isnotnull(ws_bill_customer_sk#27)) -(43) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#25, d_date#26] +(51) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#28, d_date#29] -(44) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#23] -Right keys [1]: [d_date_sk#25] +(52) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [ws_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] Join condition: None -(45) Project [codegen id : 17] -Output [2]: [ws_bill_customer_sk#24, d_date#26] -Input [4]: [ws_sold_date_sk#23, ws_bill_customer_sk#24, d_date_sk#25, d_date#26] +(53) Project [codegen id : 21] +Output [2]: [ws_bill_customer_sk#27, d_date#29] +Input [4]: [ws_sold_date_sk#26, ws_bill_customer_sk#27, d_date_sk#28, d_date#29] -(46) Exchange -Input [2]: [ws_bill_customer_sk#24, d_date#26] -Arguments: hashpartitioning(ws_bill_customer_sk#24, 5), true, [id=#27] +(54) Exchange +Input [2]: [ws_bill_customer_sk#27, d_date#29] +Arguments: hashpartitioning(ws_bill_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] -(47) Sort [codegen id : 18] -Input [2]: [ws_bill_customer_sk#24, d_date#26] -Arguments: [ws_bill_customer_sk#24 ASC NULLS FIRST], false, 0 +(55) Sort [codegen id : 22] +Input [2]: [ws_bill_customer_sk#27, d_date#29] +Arguments: [ws_bill_customer_sk#27 ASC NULLS FIRST], false, 0 -(48) ReusedExchange [Reuses operator id: 16] -Output [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] +(56) ReusedExchange [Reuses operator id: 16] +Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -(49) Sort [codegen id : 20] -Input [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 24] +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 21] -Left keys [1]: [ws_bill_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(58) SortMergeJoin [codegen id : 25] +Left keys [1]: [ws_bill_customer_sk#27] +Right keys [1]: [c_customer_sk#31] Join condition: None -(51) Project [codegen id : 21] -Output [3]: [c_last_name#30, c_first_name#29, d_date#26] -Input [5]: [ws_bill_customer_sk#24, d_date#26, c_customer_sk#28, c_first_name#29, c_last_name#30] +(59) Project [codegen id : 25] +Output [3]: [c_last_name#33, c_first_name#32, d_date#29] +Input [5]: [ws_bill_customer_sk#27, d_date#29, c_customer_sk#31, c_first_name#32, c_last_name#33] -(52) HashAggregate [codegen id : 21] -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] +(60) HashAggregate [codegen id : 25] +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Keys [3]: [c_last_name#33, c_first_name#32, d_date#29] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#30, c_first_name#29, d_date#26] +Results [3]: [c_last_name#33, c_first_name#32, d_date#29] -(53) Exchange -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), true, [id=#31] +(61) Exchange +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Arguments: hashpartitioning(c_last_name#33, c_first_name#32, d_date#29, 5), ENSURE_REQUIREMENTS, [id=#34] -(54) HashAggregate [codegen id : 22] -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] +(62) HashAggregate [codegen id : 26] +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Keys [3]: [c_last_name#33, c_first_name#32, d_date#29] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#30, c_first_name#29, d_date#26] +Results [3]: [c_last_name#33, c_first_name#32, d_date#29] -(55) Exchange -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26), 5), true, [id=#32] +(63) Exchange +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Arguments: hashpartitioning(coalesce(c_last_name#33, ), isnull(c_last_name#33), coalesce(c_first_name#32, ), isnull(c_first_name#32), coalesce(d_date#29, 0), isnull(d_date#29), 5), ENSURE_REQUIREMENTS, [id=#35] -(56) Sort [codegen id : 23] -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: [coalesce(c_last_name#30, ) ASC NULLS FIRST, isnull(c_last_name#30) ASC NULLS FIRST, coalesce(c_first_name#29, ) ASC NULLS FIRST, isnull(c_first_name#29) ASC NULLS FIRST, coalesce(d_date#26, 0) ASC NULLS FIRST, isnull(d_date#26) ASC NULLS FIRST], false, 0 +(64) Sort [codegen id : 27] +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Arguments: [coalesce(c_last_name#33, ) ASC NULLS FIRST, isnull(c_last_name#33) ASC NULLS FIRST, coalesce(c_first_name#32, ) ASC NULLS FIRST, isnull(c_first_name#32) ASC NULLS FIRST, coalesce(d_date#29, 0) ASC NULLS FIRST, isnull(d_date#29) ASC NULLS FIRST], false, 0 -(57) SortMergeJoin +(65) SortMergeJoin Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26)] +Right keys [6]: [coalesce(c_last_name#33, ), isnull(c_last_name#33), coalesce(c_first_name#32, ), isnull(c_first_name#32), coalesce(d_date#29, 0), isnull(d_date#29)] Join condition: None -(58) HashAggregate [codegen id : 24] -Input [3]: [d_date#4, c_first_name#9, c_last_name#10] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(59) Exchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), true, [id=#33] - -(60) HashAggregate [codegen id : 25] +(66) HashAggregate [codegen id : 28] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(61) HashAggregate [codegen id : 25] +(67) Exchange Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(62) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(63) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#36] -(64) HashAggregate [codegen id : 25] +(68) HashAggregate [codegen id : 29] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(69) HashAggregate [codegen id : 29] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [1]: [count#35] +Aggregate Attributes [1]: [count#37] +Results [1]: [count#38] -(66) Exchange -Input [1]: [count#35] -Arguments: SinglePartition, true, [id=#36] +(70) Exchange +Input [1]: [count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] -(67) HashAggregate [codegen id : 26] -Input [1]: [count#35] +(71) HashAggregate [codegen id : 30] +Input [1]: [count#38] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [1]: [count(1)#37 AS count(1)#38] +Aggregate Attributes [1]: [count(1)#40] +Results [1]: [count(1)#40 AS count(1)#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index 8dd59340cf069..b9626aeabf652 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -1,117 +1,129 @@ -WholeStageCodegen (26) +WholeStageCodegen (30) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (25) + WholeStageCodegen (29) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (28) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (24) - HashAggregate [c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (19) + Sort [c_last_name,c_first_name,d_date] InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (18) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #4 + WholeStageCodegen (17) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (8) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #5 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #7 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #9 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (16) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (15) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #12 + WholeStageCodegen (10) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #8 + InputAdapter + WholeStageCodegen (13) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 + WholeStageCodegen (27) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #13 + WholeStageCodegen (26) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #14 + WholeStageCodegen (25) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] + WholeStageCodegen (22) + Sort [ws_bill_customer_sk] InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (21) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [d_date_sk,d_date] #8 InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (24) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 377bd36f520eb..9f664e7f117d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -1,71 +1,75 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftAnti (57) - :- SortMergeJoin LeftAnti (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (71) ++- Exchange (70) + +- * HashAggregate (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- SortMergeJoin LeftAnti (65) + :- * Sort (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- Exchange (44) + : +- * HashAggregate (43) + : +- SortMergeJoin LeftAnti (42) + : :- * Sort (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * SortMergeJoin Inner (35) + : :- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Filter (27) + : : : +- * ColumnarToRow (26) + : : : +- Scan parquet default.catalog_sales (25) + : : +- ReusedExchange (28) + : +- * Sort (34) + : +- ReusedExchange (33) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * SortMergeJoin Inner (58) + :- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Filter (50) + : : +- * ColumnarToRow (49) + : : +- Scan parquet default.web_sales (48) + : +- ReusedExchange (51) + +- * Sort (57) + +- ReusedExchange (56) (1) Scan parquet default.store_sales @@ -115,7 +119,7 @@ Input [4]: [ss_sold_date_sk#1, ss_customer_sk#2, d_date_sk#3, d_date#4] (11) Exchange Input [2]: [ss_customer_sk#2, d_date#4] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#7] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] (12) Sort [codegen id : 3] Input [2]: [ss_customer_sk#2, d_date#4] @@ -137,7 +141,7 @@ Condition : isnotnull(c_customer_sk#8) (16) Exchange Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(c_customer_sk#8, 5), true, [id=#11] +Arguments: hashpartitioning(c_customer_sk#8, 5), ENSURE_REQUIREMENTS, [id=#11] (17) Sort [codegen id : 5] Input [3]: [c_customer_sk#8, c_first_name#9, c_last_name#10] @@ -149,240 +153,256 @@ Right keys [1]: [c_customer_sk#8] Join condition: None (19) Project [codegen id : 6] -Output [3]: [d_date#4, c_first_name#9, c_last_name#10] +Output [3]: [c_last_name#10, c_first_name#9, d_date#4] Input [5]: [ss_customer_sk#2, d_date#4, c_customer_sk#8, c_first_name#9, c_last_name#10] -(20) Exchange -Input [3]: [d_date#4, c_first_name#9, c_last_name#10] -Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), true, [id=#12] +(20) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(21) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#12] + +(22) HashAggregate [codegen id : 7] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(23) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), ENSURE_REQUIREMENTS, [id=#13] -(21) Sort [codegen id : 7] -Input [3]: [d_date#4, c_first_name#9, c_last_name#10] +(24) Sort [codegen id : 8] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Arguments: [coalesce(c_last_name#10, ) ASC NULLS FIRST, isnull(c_last_name#10) ASC NULLS FIRST, coalesce(c_first_name#9, ) ASC NULLS FIRST, isnull(c_first_name#9) ASC NULLS FIRST, coalesce(d_date#4, 0) ASC NULLS FIRST, isnull(d_date#4) ASC NULLS FIRST], false, 0 -(22) Scan parquet default.catalog_sales -Output [2]: [cs_sold_date_sk#13, cs_bill_customer_sk#14] +(25) Scan parquet default.catalog_sales +Output [2]: [cs_sold_date_sk#14, cs_bill_customer_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 9] -Input [2]: [cs_sold_date_sk#13, cs_bill_customer_sk#14] +(26) ColumnarToRow [codegen id : 10] +Input [2]: [cs_sold_date_sk#14, cs_bill_customer_sk#15] -(24) Filter [codegen id : 9] -Input [2]: [cs_sold_date_sk#13, cs_bill_customer_sk#14] -Condition : (isnotnull(cs_sold_date_sk#13) AND isnotnull(cs_bill_customer_sk#14)) +(27) Filter [codegen id : 10] +Input [2]: [cs_sold_date_sk#14, cs_bill_customer_sk#15] +Condition : (isnotnull(cs_sold_date_sk#14) AND isnotnull(cs_bill_customer_sk#15)) -(25) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#15, d_date#16] +(28) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#16, d_date#17] -(26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] +(29) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] Join condition: None -(27) Project [codegen id : 9] -Output [2]: [cs_bill_customer_sk#14, d_date#16] -Input [4]: [cs_sold_date_sk#13, cs_bill_customer_sk#14, d_date_sk#15, d_date#16] +(30) Project [codegen id : 10] +Output [2]: [cs_bill_customer_sk#15, d_date#17] +Input [4]: [cs_sold_date_sk#14, cs_bill_customer_sk#15, d_date_sk#16, d_date#17] -(28) Exchange -Input [2]: [cs_bill_customer_sk#14, d_date#16] -Arguments: hashpartitioning(cs_bill_customer_sk#14, 5), true, [id=#17] +(31) Exchange +Input [2]: [cs_bill_customer_sk#15, d_date#17] +Arguments: hashpartitioning(cs_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] -(29) Sort [codegen id : 10] -Input [2]: [cs_bill_customer_sk#14, d_date#16] -Arguments: [cs_bill_customer_sk#14 ASC NULLS FIRST], false, 0 +(32) Sort [codegen id : 11] +Input [2]: [cs_bill_customer_sk#15, d_date#17] +Arguments: [cs_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(30) ReusedExchange [Reuses operator id: 16] -Output [3]: [c_customer_sk#18, c_first_name#19, c_last_name#20] +(33) ReusedExchange [Reuses operator id: 16] +Output [3]: [c_customer_sk#19, c_first_name#20, c_last_name#21] -(31) Sort [codegen id : 12] -Input [3]: [c_customer_sk#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#18 ASC NULLS FIRST], false, 0 +(34) Sort [codegen id : 13] +Input [3]: [c_customer_sk#19, c_first_name#20, c_last_name#21] +Arguments: [c_customer_sk#19 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#14] -Right keys [1]: [c_customer_sk#18] +(35) SortMergeJoin [codegen id : 14] +Left keys [1]: [cs_bill_customer_sk#15] +Right keys [1]: [c_customer_sk#19] Join condition: None -(33) Project [codegen id : 13] -Output [3]: [c_last_name#20, c_first_name#19, d_date#16] -Input [5]: [cs_bill_customer_sk#14, d_date#16, c_customer_sk#18, c_first_name#19, c_last_name#20] +(36) Project [codegen id : 14] +Output [3]: [c_last_name#21, c_first_name#20, d_date#17] +Input [5]: [cs_bill_customer_sk#15, d_date#17, c_customer_sk#19, c_first_name#20, c_last_name#21] -(34) HashAggregate [codegen id : 13] -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Keys [3]: [c_last_name#20, c_first_name#19, d_date#16] +(37) HashAggregate [codegen id : 14] +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Keys [3]: [c_last_name#21, c_first_name#20, d_date#17] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#20, c_first_name#19, d_date#16] +Results [3]: [c_last_name#21, c_first_name#20, d_date#17] -(35) Exchange -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(c_last_name#20, c_first_name#19, d_date#16, 5), true, [id=#21] +(38) Exchange +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Arguments: hashpartitioning(c_last_name#21, c_first_name#20, d_date#17, 5), ENSURE_REQUIREMENTS, [id=#22] -(36) HashAggregate [codegen id : 14] -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Keys [3]: [c_last_name#20, c_first_name#19, d_date#16] +(39) HashAggregate [codegen id : 15] +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Keys [3]: [c_last_name#21, c_first_name#20, d_date#17] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#20, c_first_name#19, d_date#16] +Results [3]: [c_last_name#21, c_first_name#20, d_date#17] -(37) Exchange -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: hashpartitioning(coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16), 5), true, [id=#22] +(40) Exchange +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Arguments: hashpartitioning(coalesce(c_last_name#21, ), isnull(c_last_name#21), coalesce(c_first_name#20, ), isnull(c_first_name#20), coalesce(d_date#17, 0), isnull(d_date#17), 5), ENSURE_REQUIREMENTS, [id=#23] -(38) Sort [codegen id : 15] -Input [3]: [c_last_name#20, c_first_name#19, d_date#16] -Arguments: [coalesce(c_last_name#20, ) ASC NULLS FIRST, isnull(c_last_name#20) ASC NULLS FIRST, coalesce(c_first_name#19, ) ASC NULLS FIRST, isnull(c_first_name#19) ASC NULLS FIRST, coalesce(d_date#16, 0) ASC NULLS FIRST, isnull(d_date#16) ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 16] +Input [3]: [c_last_name#21, c_first_name#20, d_date#17] +Arguments: [coalesce(c_last_name#21, ) ASC NULLS FIRST, isnull(c_last_name#21) ASC NULLS FIRST, coalesce(c_first_name#20, ) ASC NULLS FIRST, isnull(c_first_name#20) ASC NULLS FIRST, coalesce(d_date#17, 0) ASC NULLS FIRST, isnull(d_date#17) ASC NULLS FIRST], false, 0 -(39) SortMergeJoin +(42) SortMergeJoin Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 0), isnull(d_date#16)] +Right keys [6]: [coalesce(c_last_name#21, ), isnull(c_last_name#21), coalesce(c_first_name#20, ), isnull(c_first_name#20), coalesce(d_date#17, 0), isnull(d_date#17)] Join condition: None -(40) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#23, ws_bill_customer_sk#24] +(43) HashAggregate [codegen id : 17] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(44) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#24] + +(45) HashAggregate [codegen id : 18] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#10, c_first_name#9, d_date#4] + +(46) Exchange +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4), 5), ENSURE_REQUIREMENTS, [id=#25] + +(47) Sort [codegen id : 19] +Input [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: [coalesce(c_last_name#10, ) ASC NULLS FIRST, isnull(c_last_name#10) ASC NULLS FIRST, coalesce(c_first_name#9, ) ASC NULLS FIRST, isnull(c_first_name#9) ASC NULLS FIRST, coalesce(d_date#4, 0) ASC NULLS FIRST, isnull(d_date#4) ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.web_sales +Output [2]: [ws_sold_date_sk#26, ws_bill_customer_sk#27] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 17] -Input [2]: [ws_sold_date_sk#23, ws_bill_customer_sk#24] +(49) ColumnarToRow [codegen id : 21] +Input [2]: [ws_sold_date_sk#26, ws_bill_customer_sk#27] -(42) Filter [codegen id : 17] -Input [2]: [ws_sold_date_sk#23, ws_bill_customer_sk#24] -Condition : (isnotnull(ws_sold_date_sk#23) AND isnotnull(ws_bill_customer_sk#24)) +(50) Filter [codegen id : 21] +Input [2]: [ws_sold_date_sk#26, ws_bill_customer_sk#27] +Condition : (isnotnull(ws_sold_date_sk#26) AND isnotnull(ws_bill_customer_sk#27)) -(43) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#25, d_date#26] +(51) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#28, d_date#29] -(44) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#23] -Right keys [1]: [d_date_sk#25] +(52) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [ws_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] Join condition: None -(45) Project [codegen id : 17] -Output [2]: [ws_bill_customer_sk#24, d_date#26] -Input [4]: [ws_sold_date_sk#23, ws_bill_customer_sk#24, d_date_sk#25, d_date#26] +(53) Project [codegen id : 21] +Output [2]: [ws_bill_customer_sk#27, d_date#29] +Input [4]: [ws_sold_date_sk#26, ws_bill_customer_sk#27, d_date_sk#28, d_date#29] -(46) Exchange -Input [2]: [ws_bill_customer_sk#24, d_date#26] -Arguments: hashpartitioning(ws_bill_customer_sk#24, 5), true, [id=#27] +(54) Exchange +Input [2]: [ws_bill_customer_sk#27, d_date#29] +Arguments: hashpartitioning(ws_bill_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] -(47) Sort [codegen id : 18] -Input [2]: [ws_bill_customer_sk#24, d_date#26] -Arguments: [ws_bill_customer_sk#24 ASC NULLS FIRST], false, 0 +(55) Sort [codegen id : 22] +Input [2]: [ws_bill_customer_sk#27, d_date#29] +Arguments: [ws_bill_customer_sk#27 ASC NULLS FIRST], false, 0 -(48) ReusedExchange [Reuses operator id: 16] -Output [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] +(56) ReusedExchange [Reuses operator id: 16] +Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -(49) Sort [codegen id : 20] -Input [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 24] +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 21] -Left keys [1]: [ws_bill_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(58) SortMergeJoin [codegen id : 25] +Left keys [1]: [ws_bill_customer_sk#27] +Right keys [1]: [c_customer_sk#31] Join condition: None -(51) Project [codegen id : 21] -Output [3]: [c_last_name#30, c_first_name#29, d_date#26] -Input [5]: [ws_bill_customer_sk#24, d_date#26, c_customer_sk#28, c_first_name#29, c_last_name#30] +(59) Project [codegen id : 25] +Output [3]: [c_last_name#33, c_first_name#32, d_date#29] +Input [5]: [ws_bill_customer_sk#27, d_date#29, c_customer_sk#31, c_first_name#32, c_last_name#33] -(52) HashAggregate [codegen id : 21] -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] +(60) HashAggregate [codegen id : 25] +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Keys [3]: [c_last_name#33, c_first_name#32, d_date#29] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#30, c_first_name#29, d_date#26] +Results [3]: [c_last_name#33, c_first_name#32, d_date#29] -(53) Exchange -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), true, [id=#31] +(61) Exchange +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Arguments: hashpartitioning(c_last_name#33, c_first_name#32, d_date#29, 5), ENSURE_REQUIREMENTS, [id=#34] -(54) HashAggregate [codegen id : 22] -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] +(62) HashAggregate [codegen id : 26] +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Keys [3]: [c_last_name#33, c_first_name#32, d_date#29] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#30, c_first_name#29, d_date#26] +Results [3]: [c_last_name#33, c_first_name#32, d_date#29] -(55) Exchange -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26), 5), true, [id=#32] +(63) Exchange +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Arguments: hashpartitioning(coalesce(c_last_name#33, ), isnull(c_last_name#33), coalesce(c_first_name#32, ), isnull(c_first_name#32), coalesce(d_date#29, 0), isnull(d_date#29), 5), ENSURE_REQUIREMENTS, [id=#35] -(56) Sort [codegen id : 23] -Input [3]: [c_last_name#30, c_first_name#29, d_date#26] -Arguments: [coalesce(c_last_name#30, ) ASC NULLS FIRST, isnull(c_last_name#30) ASC NULLS FIRST, coalesce(c_first_name#29, ) ASC NULLS FIRST, isnull(c_first_name#29) ASC NULLS FIRST, coalesce(d_date#26, 0) ASC NULLS FIRST, isnull(d_date#26) ASC NULLS FIRST], false, 0 +(64) Sort [codegen id : 27] +Input [3]: [c_last_name#33, c_first_name#32, d_date#29] +Arguments: [coalesce(c_last_name#33, ) ASC NULLS FIRST, isnull(c_last_name#33) ASC NULLS FIRST, coalesce(c_first_name#32, ) ASC NULLS FIRST, isnull(c_first_name#32) ASC NULLS FIRST, coalesce(d_date#29, 0) ASC NULLS FIRST, isnull(d_date#29) ASC NULLS FIRST], false, 0 -(57) SortMergeJoin +(65) SortMergeJoin Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#4, 0), isnull(d_date#4)] -Right keys [6]: [coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 0), isnull(d_date#26)] +Right keys [6]: [coalesce(c_last_name#33, ), isnull(c_last_name#33), coalesce(c_first_name#32, ), isnull(c_first_name#32), coalesce(d_date#29, 0), isnull(d_date#29)] Join condition: None -(58) HashAggregate [codegen id : 24] -Input [3]: [d_date#4, c_first_name#9, c_last_name#10] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(59) Exchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), true, [id=#33] - -(60) HashAggregate [codegen id : 25] +(66) HashAggregate [codegen id : 28] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#10, c_first_name#9, d_date#4] -(61) HashAggregate [codegen id : 25] +(67) Exchange Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(62) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(63) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] +Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#36] -(64) HashAggregate [codegen id : 25] +(68) HashAggregate [codegen id : 29] Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(69) HashAggregate [codegen id : 29] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [1]: [count#35] +Aggregate Attributes [1]: [count#37] +Results [1]: [count#38] -(66) Exchange -Input [1]: [count#35] -Arguments: SinglePartition, true, [id=#36] +(70) Exchange +Input [1]: [count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] -(67) HashAggregate [codegen id : 26] -Input [1]: [count#35] +(71) HashAggregate [codegen id : 30] +Input [1]: [count#38] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [1]: [count(1)#37 AS count(1)#38] +Aggregate Attributes [1]: [count(1)#40] +Results [1]: [count(1)#40 AS count(1)#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt index 8dd59340cf069..b9626aeabf652 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt @@ -1,117 +1,129 @@ -WholeStageCodegen (26) +WholeStageCodegen (30) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (25) + WholeStageCodegen (29) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (28) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (24) - HashAggregate [c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (19) + Sort [c_last_name,c_first_name,d_date] InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (18) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #4 + WholeStageCodegen (17) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (8) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #5 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #7 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #9 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (16) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (15) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #12 + WholeStageCodegen (10) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #8 + InputAdapter + WholeStageCodegen (13) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 + WholeStageCodegen (27) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #13 + WholeStageCodegen (26) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #14 + WholeStageCodegen (25) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] + WholeStageCodegen (22) + Sort [ws_bill_customer_sk] InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (21) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [d_date_sk,d_date] #8 InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (24) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index dad6098ce4685..5be4414a09f6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -1,114 +1,117 @@ == Physical Plan == -TakeOrderedAndProject (110) -+- * BroadcastHashJoin Inner BuildRight (109) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (113) ++- * BroadcastHashJoin Inner BuildRight (112) + :- * Project (90) + : +- * Filter (89) + : +- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- SortMergeJoin LeftSemi (67) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (66) + : : : +- Exchange (65) + : : : +- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - +- BroadcastExchange (108) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- SortMergeJoin LeftSemi (92) - : : :- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (91) - : : +- ReusedExchange (90) - : +- BroadcastExchange (97) - : +- * Project (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet default.date_dim (93) - +- ReusedExchange (100) + : : : +- BroadcastExchange (62) + : : : +- * HashAggregate (61) + : : : +- Exchange (60) + : : : +- * HashAggregate (59) + : : : +- SortMergeJoin LeftSemi (58) + : : : :- * Sort (46) + : : : : +- Exchange (45) + : : : : +- * HashAggregate (44) + : : : : +- Exchange (43) + : : : : +- * HashAggregate (42) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (57) + : : : +- Exchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : :- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.web_sales (47) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (72) + : : +- * Project (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.date_dim (68) + : +- BroadcastExchange (83) + : +- SortMergeJoin LeftSemi (82) + : :- * Sort (79) + : : +- Exchange (78) + : : +- * Filter (77) + : : +- * ColumnarToRow (76) + : : +- Scan parquet default.item (75) + : +- * Sort (81) + : +- ReusedExchange (80) + +- BroadcastExchange (111) + +- * Project (110) + +- * Filter (109) + +- * HashAggregate (108) + +- Exchange (107) + +- * HashAggregate (106) + +- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (102) + : +- * BroadcastHashJoin Inner BuildRight (101) + : :- SortMergeJoin LeftSemi (95) + : : :- * Sort (92) + : : : +- ReusedExchange (91) + : : +- * Sort (94) + : : +- ReusedExchange (93) + : +- BroadcastExchange (100) + : +- * Project (99) + : +- * Filter (98) + : +- * ColumnarToRow (97) + : +- Scan parquet default.date_dim (96) + +- ReusedExchange (103) (1) Scan parquet default.store_sales @@ -127,7 +130,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) (4) Exchange Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#5] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#5] (5) Sort [codegen id : 2] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] @@ -140,10 +143,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 17] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(8) Filter [codegen id : 17] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) @@ -221,7 +224,7 @@ Input [5]: [ss_item_sk#2, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id (25) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), true, [id=#17] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#17] (26) Sort [codegen id : 6] Input [3]: [brand_id#14, class_id#15, category_id#16] @@ -282,7 +285,7 @@ Input [5]: [cs_item_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_i (39) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), true, [id=#21] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#21] (40) Sort [codegen id : 10] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] @@ -293,518 +296,530 @@ Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(42) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(43) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#22] + +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(45) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#23] + +(46) Sort [codegen id : 13] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: [coalesce(brand_id#14, 0) ASC NULLS FIRST, isnull(brand_id#14) ASC NULLS FIRST, coalesce(class_id#15, 0) ASC NULLS FIRST, isnull(class_id#15) ASC NULLS FIRST, coalesce(category_id#16, 0) ASC NULLS FIRST, isnull(category_id#16) ASC NULLS FIRST], false, 0 + +(47) Scan parquet default.web_sales +Output [2]: [ws_sold_date_sk#24, ws_item_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] -(44) Filter [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] -Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) +(49) Filter [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] +Condition : (isnotnull(ws_item_sk#25) AND isnotnull(ws_sold_date_sk#24)) -(45) ReusedExchange [Reuses operator id: 16] +(50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#10] -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#22] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(47) Project [codegen id : 13] -Output [1]: [ws_item_sk#23] -Input [3]: [ws_sold_date_sk#22, ws_item_sk#23, d_date_sk#10] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#25] +Input [3]: [ws_sold_date_sk#24, ws_item_sk#25, d_date_sk#10] -(48) ReusedExchange [Reuses operator id: 36] +(53) ReusedExchange [Reuses operator id: 36] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#23] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#6] Join condition: None -(50) Project [codegen id : 13] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_item_sk#23, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_item_sk#25, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(51) Exchange +(56) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), true, [id=#24] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#26] -(52) Sort [codegen id : 14] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [coalesce(i_brand_id#7, 0) ASC NULLS FIRST, isnull(i_brand_id#7) ASC NULLS FIRST, coalesce(i_class_id#8, 0) ASC NULLS FIRST, isnull(i_class_id#8) ASC NULLS FIRST, coalesce(i_category_id#9, 0) ASC NULLS FIRST, isnull(i_category_id#9) ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(58) SortMergeJoin Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16)] Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(54) HashAggregate [codegen id : 15] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(55) Exchange -Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), true, [id=#25] - -(56) HashAggregate [codegen id : 16] +(60) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#27] -(58) HashAggregate [codegen id : 16] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(59) BroadcastExchange +(62) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#28] -(60) BroadcastHashJoin [codegen id : 17] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#6 AS ss_item_sk#29] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange -Input [1]: [ss_item_sk#27] -Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] +(65) Exchange +Input [1]: [ss_item_sk#29] +Arguments: hashpartitioning(ss_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] -(63) Sort [codegen id : 18] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(67) SortMergeJoin Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(65) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_week_seq#29] +(68) Scan parquet default.date_dim +Output [2]: [d_date_sk#10, d_week_seq#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] -Input [2]: [d_date_sk#10, d_week_seq#29] +(69) ColumnarToRow [codegen id : 22] +Input [2]: [d_date_sk#10, d_week_seq#31] -(67) Filter [codegen id : 19] -Input [2]: [d_date_sk#10, d_week_seq#29] -Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#30, [id=#31])) AND isnotnull(d_date_sk#10)) +(70) Filter [codegen id : 22] +Input [2]: [d_date_sk#10, d_week_seq#31] +Condition : ((isnotnull(d_week_seq#31) AND (d_week_seq#31 = Subquery scalar-subquery#32, [id=#33])) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_week_seq#29] +Input [2]: [d_date_sk#10, d_week_seq#31] -(69) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(70) BroadcastHashJoin [codegen id : 38] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(75) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(75) Exchange +(78) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#33] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#35] -(76) Sort [codegen id : 21] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(80) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(78) Sort [codegen id : 37] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(82) SortMergeJoin Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(80) BroadcastExchange +(83) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] -(81) BroadcastHashJoin [codegen id : 38] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] +Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#40, isEmpty#41, count#42] -(84) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#41] +(87) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#40, isEmpty#41, count#42] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#43] -(85) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] +(88) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#40, isEmpty#41, count#42] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48] -(86) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) +(89) Filter [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(87) Project [codegen id : 78] -Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +(90) Project [codegen id : 90] +Output [6]: [store AS channel#51, i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#48] -(88) ReusedExchange [Reuses operator id: 4] +(91) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(89) Sort [codegen id : 40] +(92) Sort [codegen id : 46] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(93) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(91) Sort [codegen id : 56] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(94) Sort [codegen id : 65] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(95) SortMergeJoin Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(93) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_week_seq#29] +(96) Scan parquet default.date_dim +Output [2]: [d_date_sk#10, d_week_seq#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 57] -Input [2]: [d_date_sk#10, d_week_seq#29] +(97) ColumnarToRow [codegen id : 66] +Input [2]: [d_date_sk#10, d_week_seq#31] -(95) Filter [codegen id : 57] -Input [2]: [d_date_sk#10, d_week_seq#29] -Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#10)) +(98) Filter [codegen id : 66] +Input [2]: [d_date_sk#10, d_week_seq#31] +Condition : ((isnotnull(d_week_seq#31) AND (d_week_seq#31 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 57] +(99) Project [codegen id : 66] Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_week_seq#29] +Input [2]: [d_date_sk#10, d_week_seq#31] -(97) BroadcastExchange +(100) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] -(98) BroadcastHashJoin [codegen id : 76] +(101) BroadcastHashJoin [codegen id : 88] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(99) Project [codegen id : 76] +(102) Project [codegen id : 88] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(100) ReusedExchange [Reuses operator id: 80] -Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(103) ReusedExchange [Reuses operator id: 83] +Output [4]: [i_item_sk#55, i_brand_id#56, i_class_id#57, i_category_id#58] -(101) BroadcastHashJoin [codegen id : 76] +(104) BroadcastHashJoin [codegen id : 88] Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#53] +Right keys [1]: [i_item_sk#55] Join condition: None -(102) Project [codegen id : 76] -Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(105) Project [codegen id : 88] +Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#56, i_class_id#57, i_category_id#58] +Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#55, i_brand_id#56, i_class_id#57, i_category_id#58] -(103) HashAggregate [codegen id : 76] -Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(106) HashAggregate [codegen id : 88] +Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#56, i_class_id#57, i_category_id#58] +Keys [3]: [i_brand_id#56, i_class_id#57, i_category_id#58] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum#62, isEmpty#63, count#64] -(104) Exchange -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), true, [id=#63] +(107) Exchange +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#56, i_class_id#57, i_category_id#58, 5), ENSURE_REQUIREMENTS, [id=#65] -(105) HashAggregate [codegen id : 77] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(108) HashAggregate [codegen id : 89] +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#56, i_class_id#57, i_category_id#58] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70] -(106) Filter [codegen id : 77] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) +(109) Filter [codegen id : 89] +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(107) Project [codegen id : 77] -Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] +(110) Project [codegen id : 89] +Output [6]: [store AS channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] +Input [6]: [i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#70] -(108) BroadcastExchange -Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +(111) BroadcastExchange +Input [6]: [channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#72] -(109) BroadcastHashJoin [codegen id : 78] +(112) BroadcastHashJoin [codegen id : 90] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +Right keys [3]: [i_brand_id#56, i_class_id#57, i_category_id#58] Join condition: None -(110) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] +(113) TakeOrderedAndProject +Input [12]: [channel#51, i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#51, i_brand_id#7, i_class_id#8, i_category_id#9, sales#46, number_sales#47, channel#71, i_brand_id#56, i_class_id#57, i_category_id#58, sales#68, number_sales#69] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (136) -+- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.store_sales (111) - : +- BroadcastExchange (118) - : +- * Project (117) - : +- * Filter (116) - : +- * ColumnarToRow (115) - : +- Scan parquet default.date_dim (114) - :- * Project (126) - : +- * BroadcastHashJoin Inner BuildRight (125) - : :- * Filter (123) - : : +- * ColumnarToRow (122) - : : +- Scan parquet default.catalog_sales (121) - : +- ReusedExchange (124) - +- * Project (132) - +- * BroadcastHashJoin Inner BuildRight (131) - :- * Filter (129) - : +- * ColumnarToRow (128) - : +- Scan parquet default.web_sales (127) - +- ReusedExchange (130) - - -(111) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (139) ++- Exchange (138) + +- * HashAggregate (137) + +- Union (136) + :- * Project (123) + : +- * BroadcastHashJoin Inner BuildRight (122) + : :- * Filter (116) + : : +- * ColumnarToRow (115) + : : +- Scan parquet default.store_sales (114) + : +- BroadcastExchange (121) + : +- * Project (120) + : +- * Filter (119) + : +- * ColumnarToRow (118) + : +- Scan parquet default.date_dim (117) + :- * Project (129) + : +- * BroadcastHashJoin Inner BuildRight (128) + : :- * Filter (126) + : : +- * ColumnarToRow (125) + : : +- Scan parquet default.catalog_sales (124) + : +- ReusedExchange (127) + +- * Project (135) + +- * BroadcastHashJoin Inner BuildRight (134) + :- * Filter (132) + : +- * ColumnarToRow (131) + : +- Scan parquet default.web_sales (130) + +- ReusedExchange (133) + + +(114) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(115) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(113) Filter [codegen id : 2] +(116) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(114) Scan parquet default.date_dim +(117) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(115) ColumnarToRow [codegen id : 1] +(118) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(116) Filter [codegen id : 1] +(119) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(117) Project [codegen id : 1] +(120) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(118) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] -(119) BroadcastHashJoin [codegen id : 2] +(122) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(120) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#72, ss_list_price#4 AS list_price#73] +(123) Project [codegen id : 2] +Output [2]: [ss_quantity#3 AS quantity#74, ss_list_price#4 AS list_price#75] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(121) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] +(124) Scan parquet default.catalog_sales +Output [3]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] +(125) ColumnarToRow [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77] -(123) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] +(126) Filter [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77] Condition : isnotnull(cs_sold_date_sk#18) -(124) ReusedExchange [Reuses operator id: 118] +(127) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#10] -(125) BroadcastHashJoin [codegen id : 4] +(128) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(126) Project [codegen id : 4] -Output [2]: [cs_quantity#74 AS quantity#76, cs_list_price#75 AS list_price#77] -Input [4]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75, d_date_sk#10] +(129) Project [codegen id : 4] +Output [2]: [cs_quantity#76 AS quantity#78, cs_list_price#77 AS list_price#79] +Input [4]: [cs_sold_date_sk#18, cs_quantity#76, cs_list_price#77, d_date_sk#10] -(127) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] +(130) Scan parquet default.web_sales +Output [3]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] +(131) ColumnarToRow [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81] -(129) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] -Condition : isnotnull(ws_sold_date_sk#22) +(132) Filter [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81] +Condition : isnotnull(ws_sold_date_sk#24) -(130) ReusedExchange [Reuses operator id: 118] +(133) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#10] -(131) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#22] +(134) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(132) Project [codegen id : 6] -Output [2]: [ws_quantity#78 AS quantity#80, ws_list_price#79 AS list_price#81] -Input [4]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79, d_date_sk#10] +(135) Project [codegen id : 6] +Output [2]: [ws_quantity#80 AS quantity#82, ws_list_price#81 AS list_price#83] +Input [4]: [ws_sold_date_sk#24, ws_quantity#80, ws_list_price#81, d_date_sk#10] -(133) Union +(136) Union -(134) HashAggregate [codegen id : 7] -Input [2]: [quantity#72, list_price#73] +(137) HashAggregate [codegen id : 7] +Input [2]: [quantity#74, list_price#75] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#82, count#83] -Results [2]: [sum#84, count#85] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] -(135) Exchange -Input [2]: [sum#84, count#85] -Arguments: SinglePartition, true, [id=#86] +(138) Exchange +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] -(136) HashAggregate [codegen id : 8] -Input [2]: [sum#84, count#85] +(139) HashAggregate [codegen id : 8] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87 AS average_sales#88] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#89] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#89 AS average_sales#90] -Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* Project (140) -+- * Filter (139) - +- * ColumnarToRow (138) - +- Scan parquet default.date_dim (137) +Subquery:2 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#32, [id=#33] +* Project (143) ++- * Filter (142) + +- * ColumnarToRow (141) + +- Scan parquet default.date_dim (140) -(137) Scan parquet default.date_dim -Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(140) Scan parquet default.date_dim +Output [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(141) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] -(139) Filter [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) +(142) Filter [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#91)) AND isnotnull(d_dom#92)) AND (d_year#11 = 1999)) AND (d_moy#91 = 12)) AND (d_dom#92 = 16)) -(140) Project [codegen id : 1] -Output [1]: [d_week_seq#29] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(143) Project [codegen id : 1] +Output [1]: [d_week_seq#31] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] +Subquery:3 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:4 Hosting operator id = 98 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* Project (147) ++- * Filter (146) + +- * ColumnarToRow (145) + +- Scan parquet default.date_dim (144) -(141) Scan parquet default.date_dim -Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(144) Scan parquet default.date_dim +Output [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(145) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] -(143) Filter [codegen id : 1] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1998)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) +(146) Filter [codegen id : 1] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#91)) AND isnotnull(d_dom#92)) AND (d_year#11 = 1998)) AND (d_moy#91 = 12)) AND (d_dom#92 = 16)) -(144) Project [codegen id : 1] -Output [1]: [d_week_seq#29] -Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] +(147) Project [codegen id : 1] +Output [1]: [d_week_seq#31] +Input [4]: [d_week_seq#31, d_year#11, d_moy#91, d_dom#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index d6b8ba4395d2e..f6ebac11d2742 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (78) + WholeStageCodegen (90) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -7,7 +7,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter - Exchange #15 + Exchange #17 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter - BroadcastExchange #16 + BroadcastExchange #18 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -35,7 +35,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter - ReusedExchange [d_date_sk] #16 + ReusedExchange [d_date_sk] #18 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -44,11 +44,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter - ReusedExchange [d_date_sk] #16 + ReusedExchange [d_date_sk] #18 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (38) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -65,11 +65,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - WholeStageCodegen (18) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #3 - WholeStageCodegen (17) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -78,87 +78,96 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (16) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #9 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (19) + BroadcastExchange #14 + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #1 @@ -172,31 +181,31 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #13 + BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (21) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #14 - WholeStageCodegen (20) + Exchange [i_item_sk] #16 + WholeStageCodegen (23) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (37) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (77) + BroadcastExchange #19 + WholeStageCodegen (89) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #18 - WholeStageCodegen (76) + Exchange [i_brand_id,i_class_id,i_category_id] #20 + WholeStageCodegen (88) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -204,17 +213,17 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (40) + WholeStageCodegen (46) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] #2 - WholeStageCodegen (56) + WholeStageCodegen (65) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - BroadcastExchange #19 - WholeStageCodegen (57) + BroadcastExchange #21 + WholeStageCodegen (66) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -228,4 +237,4 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index 7be9447d16b45..2e56482b73b7a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -1,214 +1,217 @@ == Physical Plan == -TakeOrderedAndProject (210) -+- * HashAggregate (209) - +- Exchange (208) - +- * HashAggregate (207) - +- Union (206) - :- * HashAggregate (129) - : +- Exchange (128) - : +- * HashAggregate (127) - : +- Union (126) - : :- * Project (87) - : : +- * Filter (86) - : : +- * HashAggregate (85) - : : +- Exchange (84) - : : +- * HashAggregate (83) - : : +- * Project (82) - : : +- * BroadcastHashJoin Inner BuildRight (81) - : : :- * Project (71) - : : : +- * BroadcastHashJoin Inner BuildRight (70) - : : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (213) ++- * HashAggregate (212) + +- Exchange (211) + +- * HashAggregate (210) + +- Union (209) + :- * HashAggregate (132) + : +- Exchange (131) + : +- * HashAggregate (130) + : +- Union (129) + : :- * Project (90) + : : +- * Filter (89) + : : +- * HashAggregate (88) + : : +- Exchange (87) + : : +- * HashAggregate (86) + : : +- * Project (85) + : : +- * BroadcastHashJoin Inner BuildRight (84) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- SortMergeJoin LeftSemi (67) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (63) - : : : : +- Exchange (62) - : : : : +- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : +- * Sort (66) + : : : : +- Exchange (65) + : : : : +- * Project (64) + : : : : +- * BroadcastHashJoin Inner BuildRight (63) : : : : :- * Filter (8) : : : : : +- * ColumnarToRow (7) : : : : : +- Scan parquet default.item (6) - : : : : +- BroadcastExchange (59) - : : : : +- * HashAggregate (58) - : : : : +- * HashAggregate (57) - : : : : +- * HashAggregate (56) - : : : : +- Exchange (55) - : : : : +- * HashAggregate (54) - : : : : +- SortMergeJoin LeftSemi (53) - : : : : :- SortMergeJoin LeftSemi (41) - : : : : : :- * Sort (26) - : : : : : : +- Exchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (18) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : :- * Filter (11) - : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : +- BroadcastExchange (16) - : : : : : : : +- * Project (15) - : : : : : : : +- * Filter (14) - : : : : : : : +- * ColumnarToRow (13) - : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : +- BroadcastExchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * Project (38) - : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : :- * Project (32) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : :- * Filter (29) - : : : : : : : +- * ColumnarToRow (28) - : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : +- ReusedExchange (30) - : : : : : +- BroadcastExchange (36) - : : : : : +- * Filter (35) - : : : : : +- * ColumnarToRow (34) - : : : : : +- Scan parquet default.item (33) - : : : : +- * Sort (52) - : : : : +- Exchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * Project (47) - : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : :- * Filter (44) - : : : : : : +- * ColumnarToRow (43) - : : : : : : +- Scan parquet default.web_sales (42) - : : : : : +- ReusedExchange (45) - : : : : +- ReusedExchange (48) - : : : +- BroadcastExchange (69) - : : : +- * Project (68) - : : : +- * Filter (67) - : : : +- * ColumnarToRow (66) - : : : +- Scan parquet default.date_dim (65) - : : +- BroadcastExchange (80) - : : +- SortMergeJoin LeftSemi (79) - : : :- * Sort (76) - : : : +- Exchange (75) - : : : +- * Filter (74) - : : : +- * ColumnarToRow (73) - : : : +- Scan parquet default.item (72) - : : +- * Sort (78) - : : +- ReusedExchange (77) - : :- * Project (106) - : : +- * Filter (105) - : : +- * HashAggregate (104) - : : +- Exchange (103) - : : +- * HashAggregate (102) - : : +- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * Project (98) - : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : :- SortMergeJoin LeftSemi (95) - : : : : :- * Sort (92) - : : : : : +- Exchange (91) - : : : : : +- * Filter (90) - : : : : : +- * ColumnarToRow (89) - : : : : : +- Scan parquet default.catalog_sales (88) - : : : : +- * Sort (94) - : : : : +- ReusedExchange (93) - : : : +- ReusedExchange (96) - : : +- ReusedExchange (99) - : +- * Project (125) - : +- * Filter (124) - : +- * HashAggregate (123) - : +- Exchange (122) - : +- * HashAggregate (121) - : +- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Project (117) - : : +- * BroadcastHashJoin Inner BuildRight (116) - : : :- SortMergeJoin LeftSemi (114) - : : : :- * Sort (111) - : : : : +- Exchange (110) - : : : : +- * Filter (109) - : : : : +- * ColumnarToRow (108) - : : : : +- Scan parquet default.web_sales (107) - : : : +- * Sort (113) - : : : +- ReusedExchange (112) - : : +- ReusedExchange (115) - : +- ReusedExchange (118) - :- * HashAggregate (148) - : +- Exchange (147) - : +- * HashAggregate (146) - : +- * HashAggregate (145) - : +- Exchange (144) - : +- * HashAggregate (143) - : +- Union (142) - : :- * Project (133) - : : +- * Filter (132) - : : +- * HashAggregate (131) - : : +- ReusedExchange (130) - : :- * Project (137) - : : +- * Filter (136) - : : +- * HashAggregate (135) - : : +- ReusedExchange (134) - : +- * Project (141) - : +- * Filter (140) - : +- * HashAggregate (139) - : +- ReusedExchange (138) - :- * HashAggregate (167) - : +- Exchange (166) - : +- * HashAggregate (165) - : +- * HashAggregate (164) - : +- Exchange (163) - : +- * HashAggregate (162) - : +- Union (161) - : :- * Project (152) - : : +- * Filter (151) - : : +- * HashAggregate (150) - : : +- ReusedExchange (149) - : :- * Project (156) - : : +- * Filter (155) - : : +- * HashAggregate (154) - : : +- ReusedExchange (153) - : +- * Project (160) - : +- * Filter (159) - : +- * HashAggregate (158) - : +- ReusedExchange (157) - :- * HashAggregate (186) - : +- Exchange (185) - : +- * HashAggregate (184) - : +- * HashAggregate (183) - : +- Exchange (182) - : +- * HashAggregate (181) - : +- Union (180) - : :- * Project (171) - : : +- * Filter (170) - : : +- * HashAggregate (169) - : : +- ReusedExchange (168) - : :- * Project (175) - : : +- * Filter (174) - : : +- * HashAggregate (173) - : : +- ReusedExchange (172) - : +- * Project (179) - : +- * Filter (178) - : +- * HashAggregate (177) - : +- ReusedExchange (176) - +- * HashAggregate (205) - +- Exchange (204) - +- * HashAggregate (203) - +- * HashAggregate (202) - +- Exchange (201) - +- * HashAggregate (200) - +- Union (199) - :- * Project (190) - : +- * Filter (189) - : +- * HashAggregate (188) - : +- ReusedExchange (187) - :- * Project (194) - : +- * Filter (193) - : +- * HashAggregate (192) - : +- ReusedExchange (191) - +- * Project (198) - +- * Filter (197) - +- * HashAggregate (196) - +- ReusedExchange (195) + : : : : +- BroadcastExchange (62) + : : : : +- * HashAggregate (61) + : : : : +- Exchange (60) + : : : : +- * HashAggregate (59) + : : : : +- SortMergeJoin LeftSemi (58) + : : : : :- * Sort (46) + : : : : : +- Exchange (45) + : : : : : +- * HashAggregate (44) + : : : : : +- Exchange (43) + : : : : : +- * HashAggregate (42) + : : : : : +- SortMergeJoin LeftSemi (41) + : : : : : :- * Sort (26) + : : : : : : +- Exchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (18) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : : :- * Filter (11) + : : : : : : : : +- * ColumnarToRow (10) + : : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : : +- BroadcastExchange (16) + : : : : : : : +- * Project (15) + : : : : : : : +- * Filter (14) + : : : : : : : +- * ColumnarToRow (13) + : : : : : : : +- Scan parquet default.date_dim (12) + : : : : : : +- BroadcastExchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (40) + : : : : : +- Exchange (39) + : : : : : +- * Project (38) + : : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : : :- * Project (32) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : :- * Filter (29) + : : : : : : : +- * ColumnarToRow (28) + : : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : : +- ReusedExchange (30) + : : : : : +- BroadcastExchange (36) + : : : : : +- * Filter (35) + : : : : : +- * ColumnarToRow (34) + : : : : : +- Scan parquet default.item (33) + : : : : +- * Sort (57) + : : : : +- Exchange (56) + : : : : +- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Project (52) + : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : :- * Filter (49) + : : : : : : +- * ColumnarToRow (48) + : : : : : : +- Scan parquet default.web_sales (47) + : : : : : +- ReusedExchange (50) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (72) + : : : +- * Project (71) + : : : +- * Filter (70) + : : : +- * ColumnarToRow (69) + : : : +- Scan parquet default.date_dim (68) + : : +- BroadcastExchange (83) + : : +- SortMergeJoin LeftSemi (82) + : : :- * Sort (79) + : : : +- Exchange (78) + : : : +- * Filter (77) + : : : +- * ColumnarToRow (76) + : : : +- Scan parquet default.item (75) + : : +- * Sort (81) + : : +- ReusedExchange (80) + : :- * Project (109) + : : +- * Filter (108) + : : +- * HashAggregate (107) + : : +- Exchange (106) + : : +- * HashAggregate (105) + : : +- * Project (104) + : : +- * BroadcastHashJoin Inner BuildRight (103) + : : :- * Project (101) + : : : +- * BroadcastHashJoin Inner BuildRight (100) + : : : :- SortMergeJoin LeftSemi (98) + : : : : :- * Sort (95) + : : : : : +- Exchange (94) + : : : : : +- * Filter (93) + : : : : : +- * ColumnarToRow (92) + : : : : : +- Scan parquet default.catalog_sales (91) + : : : : +- * Sort (97) + : : : : +- ReusedExchange (96) + : : : +- ReusedExchange (99) + : : +- ReusedExchange (102) + : +- * Project (128) + : +- * Filter (127) + : +- * HashAggregate (126) + : +- Exchange (125) + : +- * HashAggregate (124) + : +- * Project (123) + : +- * BroadcastHashJoin Inner BuildRight (122) + : :- * Project (120) + : : +- * BroadcastHashJoin Inner BuildRight (119) + : : :- SortMergeJoin LeftSemi (117) + : : : :- * Sort (114) + : : : : +- Exchange (113) + : : : : +- * Filter (112) + : : : : +- * ColumnarToRow (111) + : : : : +- Scan parquet default.web_sales (110) + : : : +- * Sort (116) + : : : +- ReusedExchange (115) + : : +- ReusedExchange (118) + : +- ReusedExchange (121) + :- * HashAggregate (151) + : +- Exchange (150) + : +- * HashAggregate (149) + : +- * HashAggregate (148) + : +- Exchange (147) + : +- * HashAggregate (146) + : +- Union (145) + : :- * Project (136) + : : +- * Filter (135) + : : +- * HashAggregate (134) + : : +- ReusedExchange (133) + : :- * Project (140) + : : +- * Filter (139) + : : +- * HashAggregate (138) + : : +- ReusedExchange (137) + : +- * Project (144) + : +- * Filter (143) + : +- * HashAggregate (142) + : +- ReusedExchange (141) + :- * HashAggregate (170) + : +- Exchange (169) + : +- * HashAggregate (168) + : +- * HashAggregate (167) + : +- Exchange (166) + : +- * HashAggregate (165) + : +- Union (164) + : :- * Project (155) + : : +- * Filter (154) + : : +- * HashAggregate (153) + : : +- ReusedExchange (152) + : :- * Project (159) + : : +- * Filter (158) + : : +- * HashAggregate (157) + : : +- ReusedExchange (156) + : +- * Project (163) + : +- * Filter (162) + : +- * HashAggregate (161) + : +- ReusedExchange (160) + :- * HashAggregate (189) + : +- Exchange (188) + : +- * HashAggregate (187) + : +- * HashAggregate (186) + : +- Exchange (185) + : +- * HashAggregate (184) + : +- Union (183) + : :- * Project (174) + : : +- * Filter (173) + : : +- * HashAggregate (172) + : : +- ReusedExchange (171) + : :- * Project (178) + : : +- * Filter (177) + : : +- * HashAggregate (176) + : : +- ReusedExchange (175) + : +- * Project (182) + : +- * Filter (181) + : +- * HashAggregate (180) + : +- ReusedExchange (179) + +- * HashAggregate (208) + +- Exchange (207) + +- * HashAggregate (206) + +- * HashAggregate (205) + +- Exchange (204) + +- * HashAggregate (203) + +- Union (202) + :- * Project (193) + : +- * Filter (192) + : +- * HashAggregate (191) + : +- ReusedExchange (190) + :- * Project (197) + : +- * Filter (196) + : +- * HashAggregate (195) + : +- ReusedExchange (194) + +- * Project (201) + +- * Filter (200) + +- * HashAggregate (199) + +- ReusedExchange (198) (1) Scan parquet default.store_sales @@ -240,10 +243,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 17] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(8) Filter [codegen id : 17] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) @@ -393,996 +396,1008 @@ Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(42) Scan parquet default.web_sales -Output [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(43) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#22] + +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Keys [3]: [brand_id#14, class_id#15, category_id#16] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#14, class_id#15, category_id#16] + +(45) Exchange +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: hashpartitioning(coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16), 5), ENSURE_REQUIREMENTS, [id=#23] + +(46) Sort [codegen id : 13] +Input [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: [coalesce(brand_id#14, 0) ASC NULLS FIRST, isnull(brand_id#14) ASC NULLS FIRST, coalesce(class_id#15, 0) ASC NULLS FIRST, isnull(class_id#15) ASC NULLS FIRST, coalesce(category_id#16, 0) ASC NULLS FIRST, isnull(category_id#16) ASC NULLS FIRST], false, 0 + +(47) Scan parquet default.web_sales +Output [2]: [ws_sold_date_sk#24, ws_item_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] -(44) Filter [codegen id : 13] -Input [2]: [ws_sold_date_sk#22, ws_item_sk#23] -Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) +(49) Filter [codegen id : 16] +Input [2]: [ws_sold_date_sk#24, ws_item_sk#25] +Condition : (isnotnull(ws_item_sk#25) AND isnotnull(ws_sold_date_sk#24)) -(45) ReusedExchange [Reuses operator id: 16] +(50) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#10] -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#22] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(47) Project [codegen id : 13] -Output [1]: [ws_item_sk#23] -Input [3]: [ws_sold_date_sk#22, ws_item_sk#23, d_date_sk#10] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#25] +Input [3]: [ws_sold_date_sk#24, ws_item_sk#25, d_date_sk#10] -(48) ReusedExchange [Reuses operator id: 36] +(53) ReusedExchange [Reuses operator id: 36] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#23] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#6] Join condition: None -(50) Project [codegen id : 13] +(55) Project [codegen id : 16] Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_item_sk#23, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_item_sk#25, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(51) Exchange +(56) Exchange Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9), 5), ENSURE_REQUIREMENTS, [id=#26] -(52) Sort [codegen id : 14] +(57) Sort [codegen id : 17] Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [coalesce(i_brand_id#7, 0) ASC NULLS FIRST, isnull(i_brand_id#7) ASC NULLS FIRST, coalesce(i_class_id#8, 0) ASC NULLS FIRST, isnull(i_class_id#8) ASC NULLS FIRST, coalesce(i_category_id#9, 0) ASC NULLS FIRST, isnull(i_category_id#9) ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(58) SortMergeJoin Left keys [6]: [coalesce(brand_id#14, 0), isnull(brand_id#14), coalesce(class_id#15, 0), isnull(class_id#15), coalesce(category_id#16, 0), isnull(category_id#16)] Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(54) HashAggregate [codegen id : 15] +(59) HashAggregate [codegen id : 18] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(55) Exchange +(60) Exchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#25] +Arguments: hashpartitioning(brand_id#14, class_id#15, category_id#16, 5), ENSURE_REQUIREMENTS, [id=#27] -(56) HashAggregate [codegen id : 16] +(61) HashAggregate [codegen id : 19] Input [3]: [brand_id#14, class_id#15, category_id#16] Keys [3]: [brand_id#14, class_id#15, category_id#16] Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(57) HashAggregate [codegen id : 16] +(62) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#28] -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange -Input [3]: [brand_id#14, class_id#15, category_id#16] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] - -(60) BroadcastHashJoin [codegen id : 17] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#6 AS ss_item_sk#29] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange -Input [1]: [ss_item_sk#27] -Arguments: hashpartitioning(ss_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#28] +(65) Exchange +Input [1]: [ss_item_sk#29] +Arguments: hashpartitioning(ss_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] -(63) Sort [codegen id : 18] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(67) SortMergeJoin Left keys [1]: [ss_item_sk#2] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(65) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_moy#29] +(68) Scan parquet default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] -Input [3]: [d_date_sk#10, d_year#11, d_moy#29] +(69) ColumnarToRow [codegen id : 22] +Input [3]: [d_date_sk#10, d_year#11, d_moy#31] -(67) Filter [codegen id : 19] -Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2000)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) +(70) Filter [codegen id : 22] +Input [3]: [d_date_sk#10, d_year#11, d_moy#31] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#31)) AND (d_year#11 = 2000)) AND (d_moy#31 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#11, d_moy#29] +Input [3]: [d_date_sk#10, d_year#11, d_moy#31] -(69) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] -(70) BroadcastHashJoin [codegen id : 38] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(75) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(78) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#31] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#33] -(76) Sort [codegen id : 21] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(80) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(78) Sort [codegen id : 37] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(82) SortMergeJoin Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(80) BroadcastExchange +(83) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] -(81) BroadcastHashJoin [codegen id : 38] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] +Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -(84) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#39] +(87) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#41] -(85) HashAggregate [codegen id : 39] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] +(88) HashAggregate [codegen id : 45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(86) Filter [codegen id : 39] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(89) Filter [codegen id : 45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(87) Project [codegen id : 39] -Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] +(90) Project [codegen id : 45] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(88) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(91) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(92) ColumnarToRow [codegen id : 46] +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] -(90) Filter [codegen id : 40] -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(93) Filter [codegen id : 46] +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -Arguments: hashpartitioning(cs_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#50] +(94) Exchange +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] +Arguments: hashpartitioning(cs_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#52] -(92) Sort [codegen id : 41] -Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] +(95) Sort [codegen id : 47] +Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(96) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(94) Sort [codegen id : 57] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 66] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(98) SortMergeJoin Left keys [1]: [cs_item_sk#19] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#29] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(99) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(100) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] -Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] -Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] +(101) Project [codegen id : 89] +Output [3]: [cs_item_sk#19, cs_quantity#50, cs_list_price#51] +Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#50, cs_list_price#51, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(102) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(103) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] -Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(104) Project [codegen id : 89] +Output [5]: [cs_quantity#50, cs_list_price#51, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [cs_item_sk#19, cs_quantity#50, cs_list_price#51, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] -Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] +(105) HashAggregate [codegen id : 89] +Input [5]: [cs_quantity#50, cs_list_price#51, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#56, isEmpty#57, count#58] -(103) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#57] +(106) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#59] -(104) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] +(107) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#56, isEmpty#57, count#58] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64] -(105) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(108) Filter [codegen id : 90] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(106) Project [codegen id : 78] -Output [6]: [catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] +(109) Project [codegen id : 90] +Output [6]: [catalog AS channel#65, i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#64] -(107) Scan parquet default.web_sales -Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] +(110) Scan parquet default.web_sales +Output [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] +(111) ColumnarToRow [codegen id : 91] +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] -(109) Filter [codegen id : 79] -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) +(112) Filter [codegen id : 91] +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Condition : (isnotnull(ws_item_sk#25) AND isnotnull(ws_sold_date_sk#24)) -(110) Exchange -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Arguments: hashpartitioning(ws_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#66] +(113) Exchange +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Arguments: hashpartitioning(ws_item_sk#25, 5), ENSURE_REQUIREMENTS, [id=#68] -(111) Sort [codegen id : 80] -Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 92] +Input [4]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Arguments: [ws_item_sk#25 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] -Output [1]: [ss_item_sk#27] +(115) ReusedExchange [Reuses operator id: 65] +Output [1]: [ss_item_sk#29] -(113) Sort [codegen id : 96] -Input [1]: [ss_item_sk#27] -Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 111] +Input [1]: [ss_item_sk#29] +Arguments: [ss_item_sk#29 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +(117) SortMergeJoin +Left keys [1]: [ws_item_sk#25] +Right keys [1]: [ss_item_sk#29] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(118) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] -Left keys [1]: [ws_sold_date_sk#22] +(119) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] -Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] -Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] +(120) Project [codegen id : 134] +Output [3]: [ws_item_sk#25, ws_quantity#66, ws_list_price#67] +Input [5]: [ws_sold_date_sk#24, ws_item_sk#25, ws_quantity#66, ws_list_price#67, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(121) ReusedExchange [Reuses operator id: 83] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] -Left keys [1]: [ws_item_sk#23] +(122) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] -Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(123) Project [codegen id : 134] +Output [5]: [ws_quantity#66, ws_list_price#67, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ws_item_sk#25, ws_quantity#66, ws_list_price#67, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] -Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] +(124) HashAggregate [codegen id : 134] +Input [5]: [ws_quantity#66, ws_list_price#67, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#72, isEmpty#73, count#74] -(122) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#73] +(125) Exchange +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#75] -(123) HashAggregate [codegen id : 117] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] +(126) HashAggregate [codegen id : 135] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#72, isEmpty#73, count#74] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] - -(124) Filter [codegen id : 117] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) - -(125) Project [codegen id : 117] -Output [6]: [web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] - -(126) Union - -(127) HashAggregate [codegen id : 118] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] -Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] -Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] - -(128) Exchange -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#86] - -(129) HashAggregate [codegen id : 119] -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#42), sum(number_sales#43)] -Aggregate Attributes [2]: [sum(sales#42)#87, sum(number_sales#43)#88] -Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#42)#87 AS sum_sales#89, sum(number_sales#43)#88 AS number_sales#90] - -(130) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] - -(131) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80] + +(127) Filter [codegen id : 135] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) + +(128) Project [codegen id : 135] +Output [6]: [web AS channel#81, i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#80] + +(129) Union + +(130) HashAggregate [codegen id : 136] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#44), partial_sum(number_sales#45)] +Aggregate Attributes [3]: [sum#82, isEmpty#83, sum#84] +Results [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#85, isEmpty#86, sum#87] + +(131) Exchange +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#85, isEmpty#86, sum#87] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#88] + +(132) HashAggregate [codegen id : 137] +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#85, isEmpty#86, sum#87] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#44), sum(number_sales#45)] +Aggregate Attributes [2]: [sum(sales#44)#89, sum(number_sales#45)#90] +Results [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#44)#89 AS sum_sales#91, sum(number_sales#45)#90 AS number_sales#92] + +(133) ReusedExchange [Reuses operator id: 87] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#93, isEmpty#94, count#95] + +(134) HashAggregate [codegen id : 182] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#93, isEmpty#94, count#95] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94, count(1)#95] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#42, count(1)#95 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#44, count(1)#97 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#98] -(132) Filter [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(135) Filter [codegen id : 182] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#98] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#98) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(133) Project [codegen id : 158] -Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] +(136) Project [codegen id : 182] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#98] -(134) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] +(137) ReusedExchange [Reuses operator id: 106] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#99, isEmpty#100, count#101] -(135) HashAggregate [codegen id : 197] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] +(138) HashAggregate [codegen id : 227] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#99, isEmpty#100, count#101] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100, count(1)#101] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#60, count(1)#101 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#62, count(1)#103 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#104] -(136) Filter [codegen id : 197] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(139) Filter [codegen id : 227] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#104] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#104) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#104 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(137) Project [codegen id : 197] -Output [6]: [catalog AS channel#103, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] +(140) Project [codegen id : 227] +Output [6]: [catalog AS channel#105, i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#104] -(138) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] +(141) ReusedExchange [Reuses operator id: 125] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#106, isEmpty#107, count#108] -(139) HashAggregate [codegen id : 236] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] +(142) HashAggregate [codegen id : 272] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#106, isEmpty#107, count#108] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#76, count(1)#108 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] - -(140) Filter [codegen id : 236] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) - -(141) Project [codegen id : 236] -Output [6]: [web AS channel#110, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] - -(142) Union - -(143) HashAggregate [codegen id : 237] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] -Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] -Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] - -(144) Exchange -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#117] - -(145) HashAggregate [codegen id : 238] -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#42), sum(number_sales#43)] -Aggregate Attributes [2]: [sum(sales#42)#118, sum(number_sales#43)#119] -Results [5]: [channel#47, i_brand_id#7, i_class_id#8, sum(sales#42)#118 AS sum_sales#89, sum(number_sales#43)#119 AS number_sales#90] - -(146) HashAggregate [codegen id : 238] -Input [5]: [channel#47, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90] -Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] -Results [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#109, count(1)#110] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#109 AS sales#78, count(1)#110 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#109 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#111] -(147) Exchange -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#126] +(143) Filter [codegen id : 272] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) + +(144) Project [codegen id : 272] +Output [6]: [web AS channel#112, i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#111] -(148) HashAggregate [codegen id : 239] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] -Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#127, sum(number_sales#90)#128] -Results [6]: [channel#47, i_brand_id#7, i_class_id#8, null AS i_category_id#129, sum(sum_sales#89)#127 AS sum(sum_sales)#130, sum(number_sales#90)#128 AS sum(number_sales)#131] +(145) Union -(149) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#132, isEmpty#133, count#134] +(146) HashAggregate [codegen id : 273] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#44), partial_sum(number_sales#45)] +Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] +Results [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#116, isEmpty#117, sum#118] -(150) HashAggregate [codegen id : 278] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#132, isEmpty#133, count#134] +(147) Exchange +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#116, isEmpty#117, sum#118] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#119] + +(148) HashAggregate [codegen id : 274] +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#116, isEmpty#117, sum#118] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#44), sum(number_sales#45)] +Aggregate Attributes [2]: [sum(sales#44)#120, sum(number_sales#45)#121] +Results [5]: [channel#49, i_brand_id#7, i_class_id#8, sum(sales#44)#120 AS sum_sales#91, sum(number_sales#45)#121 AS number_sales#92] + +(149) HashAggregate [codegen id : 274] +Input [5]: [channel#49, i_brand_id#7, i_class_id#8, sum_sales#91, number_sales#92] +Keys [3]: [channel#49, i_brand_id#7, i_class_id#8] +Functions [2]: [partial_sum(sum_sales#91), partial_sum(number_sales#92)] +Aggregate Attributes [3]: [sum#122, isEmpty#123, sum#124] +Results [6]: [channel#49, i_brand_id#7, i_class_id#8, sum#125, isEmpty#126, sum#127] + +(150) Exchange +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, sum#125, isEmpty#126, sum#127] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#128] + +(151) HashAggregate [codegen id : 275] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, sum#125, isEmpty#126, sum#127] +Keys [3]: [channel#49, i_brand_id#7, i_class_id#8] +Functions [2]: [sum(sum_sales#91), sum(number_sales#92)] +Aggregate Attributes [2]: [sum(sum_sales#91)#129, sum(number_sales#92)#130] +Results [6]: [channel#49, i_brand_id#7, i_class_id#8, null AS i_category_id#131, sum(sum_sales#91)#129 AS sum(sum_sales)#132, sum(number_sales#92)#130 AS sum(number_sales)#133] + +(152) ReusedExchange [Reuses operator id: 87] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#134, isEmpty#135, count#136] + +(153) HashAggregate [codegen id : 320] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#134, isEmpty#135, count#136] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#135, count(1)#136] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sales#42, count(1)#136 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137, count(1)#138] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sales#44, count(1)#138 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#139] -(151) Filter [codegen id : 278] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(154) Filter [codegen id : 320] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#139] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#139) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#139 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(152) Project [codegen id : 278] -Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137] +(155) Project [codegen id : 320] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#139] -(153) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#138, isEmpty#139, count#140] +(156) ReusedExchange [Reuses operator id: 106] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#140, isEmpty#141, count#142] -(154) HashAggregate [codegen id : 317] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#138, isEmpty#139, count#140] +(157) HashAggregate [codegen id : 365] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#140, isEmpty#141, count#142] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#141, count(1)#142] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sales#60, count(1)#142 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#62, count(1)#144 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#145] -(155) Filter [codegen id : 317] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(158) Filter [codegen id : 365] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#145] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(156) Project [codegen id : 317] -Output [6]: [catalog AS channel#144, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143] +(159) Project [codegen id : 365] +Output [6]: [catalog AS channel#146, i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#145] -(157) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#145, isEmpty#146, count#147] +(160) ReusedExchange [Reuses operator id: 125] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#147, isEmpty#148, count#149] -(158) HashAggregate [codegen id : 356] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#145, isEmpty#146, count#147] +(161) HashAggregate [codegen id : 410] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#147, isEmpty#148, count#149] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148, count(1)#149] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148 AS sales#76, count(1)#149 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150] - -(159) Filter [codegen id : 356] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) - -(160) Project [codegen id : 356] -Output [6]: [web AS channel#151, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150] - -(161) Union - -(162) HashAggregate [codegen id : 357] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] -Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] -Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#155, isEmpty#156, sum#157] - -(163) Exchange -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#155, isEmpty#156, sum#157] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#158] - -(164) HashAggregate [codegen id : 358] -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#155, isEmpty#156, sum#157] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#42), sum(number_sales#43)] -Aggregate Attributes [2]: [sum(sales#42)#159, sum(number_sales#43)#160] -Results [4]: [channel#47, i_brand_id#7, sum(sales#42)#159 AS sum_sales#89, sum(number_sales#43)#160 AS number_sales#90] - -(165) HashAggregate [codegen id : 358] -Input [4]: [channel#47, i_brand_id#7, sum_sales#89, number_sales#90] -Keys [2]: [channel#47, i_brand_id#7] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] -Results [5]: [channel#47, i_brand_id#7, sum#164, isEmpty#165, sum#166] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#78, count(1)#151 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#152] -(166) Exchange -Input [5]: [channel#47, i_brand_id#7, sum#164, isEmpty#165, sum#166] -Arguments: hashpartitioning(channel#47, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#167] +(162) Filter [codegen id : 410] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#152] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) + +(163) Project [codegen id : 410] +Output [6]: [web AS channel#153, i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#152] -(167) HashAggregate [codegen id : 359] -Input [5]: [channel#47, i_brand_id#7, sum#164, isEmpty#165, sum#166] -Keys [2]: [channel#47, i_brand_id#7] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#168, sum(number_sales#90)#169] -Results [6]: [channel#47, i_brand_id#7, null AS i_class_id#170, null AS i_category_id#171, sum(sum_sales#89)#168 AS sum(sum_sales)#172, sum(number_sales#90)#169 AS sum(number_sales)#173] +(164) Union -(168) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#174, isEmpty#175, count#176] +(165) HashAggregate [codegen id : 411] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#44), partial_sum(number_sales#45)] +Aggregate Attributes [3]: [sum#154, isEmpty#155, sum#156] +Results [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#157, isEmpty#158, sum#159] -(169) HashAggregate [codegen id : 398] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#174, isEmpty#175, count#176] +(166) Exchange +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#157, isEmpty#158, sum#159] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#160] + +(167) HashAggregate [codegen id : 412] +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#157, isEmpty#158, sum#159] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#44), sum(number_sales#45)] +Aggregate Attributes [2]: [sum(sales#44)#161, sum(number_sales#45)#162] +Results [4]: [channel#49, i_brand_id#7, sum(sales#44)#161 AS sum_sales#91, sum(number_sales#45)#162 AS number_sales#92] + +(168) HashAggregate [codegen id : 412] +Input [4]: [channel#49, i_brand_id#7, sum_sales#91, number_sales#92] +Keys [2]: [channel#49, i_brand_id#7] +Functions [2]: [partial_sum(sum_sales#91), partial_sum(number_sales#92)] +Aggregate Attributes [3]: [sum#163, isEmpty#164, sum#165] +Results [5]: [channel#49, i_brand_id#7, sum#166, isEmpty#167, sum#168] + +(169) Exchange +Input [5]: [channel#49, i_brand_id#7, sum#166, isEmpty#167, sum#168] +Arguments: hashpartitioning(channel#49, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#169] + +(170) HashAggregate [codegen id : 413] +Input [5]: [channel#49, i_brand_id#7, sum#166, isEmpty#167, sum#168] +Keys [2]: [channel#49, i_brand_id#7] +Functions [2]: [sum(sum_sales#91), sum(number_sales#92)] +Aggregate Attributes [2]: [sum(sum_sales#91)#170, sum(number_sales#92)#171] +Results [6]: [channel#49, i_brand_id#7, null AS i_class_id#172, null AS i_category_id#173, sum(sum_sales#91)#170 AS sum(sum_sales)#174, sum(number_sales#92)#171 AS sum(number_sales)#175] + +(171) ReusedExchange [Reuses operator id: 87] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] + +(172) HashAggregate [codegen id : 458] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177, count(1)#178] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177 AS sales#42, count(1)#178 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179, count(1)#180] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sales#44, count(1)#180 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] -(170) Filter [codegen id : 398] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(173) Filter [codegen id : 458] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(171) Project [codegen id : 398] -Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179] +(174) Project [codegen id : 458] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] -(172) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#180, isEmpty#181, count#182] +(175) ReusedExchange [Reuses operator id: 106] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] -(173) HashAggregate [codegen id : 437] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#180, isEmpty#181, count#182] +(176) HashAggregate [codegen id : 503] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183, count(1)#184] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183 AS sales#60, count(1)#184 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#185, count(1)#186] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sales#62, count(1)#186 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#187] -(174) Filter [codegen id : 437] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(177) Filter [codegen id : 503] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#187] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#187) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#187 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(175) Project [codegen id : 437] -Output [6]: [catalog AS channel#186, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185] +(178) Project [codegen id : 503] +Output [6]: [catalog AS channel#188, i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#187] -(176) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#187, isEmpty#188, count#189] +(179) ReusedExchange [Reuses operator id: 125] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] -(177) HashAggregate [codegen id : 476] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#187, isEmpty#188, count#189] +(180) HashAggregate [codegen id : 548] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#190, count(1)#191] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#190 AS sales#76, count(1)#191 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#190 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192] - -(178) Filter [codegen id : 476] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) - -(179) Project [codegen id : 476] -Output [6]: [web AS channel#193, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192] - -(180) Union - -(181) HashAggregate [codegen id : 477] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] -Aggregate Attributes [3]: [sum#194, isEmpty#195, sum#196] -Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#197, isEmpty#198, sum#199] - -(182) Exchange -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#197, isEmpty#198, sum#199] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#200] - -(183) HashAggregate [codegen id : 478] -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#197, isEmpty#198, sum#199] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#42), sum(number_sales#43)] -Aggregate Attributes [2]: [sum(sales#42)#201, sum(number_sales#43)#202] -Results [3]: [channel#47, sum(sales#42)#201 AS sum_sales#89, sum(number_sales#43)#202 AS number_sales#90] - -(184) HashAggregate [codegen id : 478] -Input [3]: [channel#47, sum_sales#89, number_sales#90] -Keys [1]: [channel#47] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#203, isEmpty#204, sum#205] -Results [4]: [channel#47, sum#206, isEmpty#207, sum#208] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#78, count(1)#193 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#194] -(185) Exchange -Input [4]: [channel#47, sum#206, isEmpty#207, sum#208] -Arguments: hashpartitioning(channel#47, 5), ENSURE_REQUIREMENTS, [id=#209] +(181) Filter [codegen id : 548] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#194] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) + +(182) Project [codegen id : 548] +Output [6]: [web AS channel#195, i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#194] -(186) HashAggregate [codegen id : 479] -Input [4]: [channel#47, sum#206, isEmpty#207, sum#208] -Keys [1]: [channel#47] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#210, sum(number_sales#90)#211] -Results [6]: [channel#47, null AS i_brand_id#212, null AS i_class_id#213, null AS i_category_id#214, sum(sum_sales#89)#210 AS sum(sum_sales)#215, sum(number_sales#90)#211 AS sum(number_sales)#216] +(183) Union -(187) ReusedExchange [Reuses operator id: 84] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#217, isEmpty#218, count#219] +(184) HashAggregate [codegen id : 549] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#44), partial_sum(number_sales#45)] +Aggregate Attributes [3]: [sum#196, isEmpty#197, sum#198] +Results [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] -(188) HashAggregate [codegen id : 518] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#217, isEmpty#218, count#219] +(185) Exchange +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#202] + +(186) HashAggregate [codegen id : 550] +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#44), sum(number_sales#45)] +Aggregate Attributes [2]: [sum(sales#44)#203, sum(number_sales#45)#204] +Results [3]: [channel#49, sum(sales#44)#203 AS sum_sales#91, sum(number_sales#45)#204 AS number_sales#92] + +(187) HashAggregate [codegen id : 550] +Input [3]: [channel#49, sum_sales#91, number_sales#92] +Keys [1]: [channel#49] +Functions [2]: [partial_sum(sum_sales#91), partial_sum(number_sales#92)] +Aggregate Attributes [3]: [sum#205, isEmpty#206, sum#207] +Results [4]: [channel#49, sum#208, isEmpty#209, sum#210] + +(188) Exchange +Input [4]: [channel#49, sum#208, isEmpty#209, sum#210] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [id=#211] + +(189) HashAggregate [codegen id : 551] +Input [4]: [channel#49, sum#208, isEmpty#209, sum#210] +Keys [1]: [channel#49] +Functions [2]: [sum(sum_sales#91), sum(number_sales#92)] +Aggregate Attributes [2]: [sum(sum_sales#91)#212, sum(number_sales#92)#213] +Results [6]: [channel#49, null AS i_brand_id#214, null AS i_class_id#215, null AS i_category_id#216, sum(sum_sales#91)#212 AS sum(sum_sales)#217, sum(number_sales#92)#213 AS sum(number_sales)#218] + +(190) ReusedExchange [Reuses operator id: 87] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#219, isEmpty#220, count#221] + +(191) HashAggregate [codegen id : 596] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#219, isEmpty#220, count#221] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220, count(1)#221] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 AS sales#42, count(1)#221 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222, count(1)#223] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222 AS sales#44, count(1)#223 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#224] -(189) Filter [codegen id : 518] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(192) Filter [codegen id : 596] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#224] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#224) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#224 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(190) Project [codegen id : 518] -Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222] +(193) Project [codegen id : 596] +Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#224] -(191) ReusedExchange [Reuses operator id: 103] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#223, isEmpty#224, count#225] +(194) ReusedExchange [Reuses operator id: 106] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#225, isEmpty#226, count#227] -(192) HashAggregate [codegen id : 557] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#223, isEmpty#224, count#225] +(195) HashAggregate [codegen id : 641] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#225, isEmpty#226, count#227] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#226, count(1)#227] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#226 AS sales#60, count(1)#227 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#226 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#228, count(1)#229] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#228 AS sales#62, count(1)#229 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#228 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#230] -(193) Filter [codegen id : 557] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) +(196) Filter [codegen id : 641] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#230] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#230) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#230 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(194) Project [codegen id : 557] -Output [6]: [catalog AS channel#229, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228] +(197) Project [codegen id : 641] +Output [6]: [catalog AS channel#231, i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#51 as decimal(12,2)))), DecimalType(18,2), true))#230] -(195) ReusedExchange [Reuses operator id: 122] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#230, isEmpty#231, count#232] +(198) ReusedExchange [Reuses operator id: 125] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#232, isEmpty#233, count#234] -(196) HashAggregate [codegen id : 596] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#230, isEmpty#231, count#232] +(199) HashAggregate [codegen id : 686] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#232, isEmpty#233, count#234] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#233, count(1)#234] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#233 AS sales#76, count(1)#234 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#233 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235] - -(197) Filter [codegen id : 596] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) - -(198) Project [codegen id : 596] -Output [6]: [web AS channel#236, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235] - -(199) Union - -(200) HashAggregate [codegen id : 597] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] -Aggregate Attributes [3]: [sum#237, isEmpty#238, sum#239] -Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#240, isEmpty#241, sum#242] - -(201) Exchange -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#240, isEmpty#241, sum#242] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#243] - -(202) HashAggregate [codegen id : 598] -Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#240, isEmpty#241, sum#242] -Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#42), sum(number_sales#43)] -Aggregate Attributes [2]: [sum(sales#42)#244, sum(number_sales#43)#245] -Results [2]: [sum(sales#42)#244 AS sum_sales#89, sum(number_sales#43)#245 AS number_sales#90] - -(203) HashAggregate [codegen id : 598] -Input [2]: [sum_sales#89, number_sales#90] -Keys: [] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] -Aggregate Attributes [3]: [sum#246, isEmpty#247, sum#248] -Results [3]: [sum#249, isEmpty#250, sum#251] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#235, count(1)#236] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#235 AS sales#78, count(1)#236 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#235 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#237] + +(200) Filter [codegen id : 686] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#237] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#237) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#237 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) + +(201) Project [codegen id : 686] +Output [6]: [web AS channel#238, i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#237] + +(202) Union + +(203) HashAggregate [codegen id : 687] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#44), partial_sum(number_sales#45)] +Aggregate Attributes [3]: [sum#239, isEmpty#240, sum#241] +Results [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#242, isEmpty#243, sum#244] (204) Exchange -Input [3]: [sum#249, isEmpty#250, sum#251] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#252] +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#242, isEmpty#243, sum#244] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#245] + +(205) HashAggregate [codegen id : 688] +Input [7]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum#242, isEmpty#243, sum#244] +Keys [4]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#44), sum(number_sales#45)] +Aggregate Attributes [2]: [sum(sales#44)#246, sum(number_sales#45)#247] +Results [2]: [sum(sales#44)#246 AS sum_sales#91, sum(number_sales#45)#247 AS number_sales#92] + +(206) HashAggregate [codegen id : 688] +Input [2]: [sum_sales#91, number_sales#92] +Keys: [] +Functions [2]: [partial_sum(sum_sales#91), partial_sum(number_sales#92)] +Aggregate Attributes [3]: [sum#248, isEmpty#249, sum#250] +Results [3]: [sum#251, isEmpty#252, sum#253] + +(207) Exchange +Input [3]: [sum#251, isEmpty#252, sum#253] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#254] -(205) HashAggregate [codegen id : 599] -Input [3]: [sum#249, isEmpty#250, sum#251] +(208) HashAggregate [codegen id : 689] +Input [3]: [sum#251, isEmpty#252, sum#253] Keys: [] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] -Aggregate Attributes [2]: [sum(sum_sales#89)#253, sum(number_sales#90)#254] -Results [6]: [null AS channel#255, null AS i_brand_id#256, null AS i_class_id#257, null AS i_category_id#258, sum(sum_sales#89)#253 AS sum(sum_sales)#259, sum(number_sales#90)#254 AS sum(number_sales)#260] +Functions [2]: [sum(sum_sales#91), sum(number_sales#92)] +Aggregate Attributes [2]: [sum(sum_sales#91)#255, sum(number_sales#92)#256] +Results [6]: [null AS channel#257, null AS i_brand_id#258, null AS i_class_id#259, null AS i_category_id#260, sum(sum_sales#91)#255 AS sum(sum_sales)#261, sum(number_sales#92)#256 AS sum(number_sales)#262] -(206) Union +(209) Union -(207) HashAggregate [codegen id : 600] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +(210) HashAggregate [codegen id : 690] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] +Keys [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] -(208) Exchange -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), ENSURE_REQUIREMENTS, [id=#261] +(211) Exchange +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] +Arguments: hashpartitioning(channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92, 5), ENSURE_REQUIREMENTS, [id=#263] -(209) HashAggregate [codegen id : 601] -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +(212) HashAggregate [codegen id : 691] +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] +Keys [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +Results [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] -(210) TakeOrderedAndProject -Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -Arguments: 100, [channel#47 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] +(213) TakeOrderedAndProject +Input [6]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] +Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#91, number_sales#92] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (240) -+- Exchange (239) - +- * HashAggregate (238) - +- Union (237) - :- * Project (220) - : +- * BroadcastHashJoin Inner BuildRight (219) - : :- * Filter (213) - : : +- * ColumnarToRow (212) - : : +- Scan parquet default.store_sales (211) - : +- BroadcastExchange (218) - : +- * Project (217) - : +- * Filter (216) - : +- * ColumnarToRow (215) - : +- Scan parquet default.date_dim (214) - :- * Project (230) - : +- * BroadcastHashJoin Inner BuildRight (229) - : :- * Filter (223) - : : +- * ColumnarToRow (222) - : : +- Scan parquet default.catalog_sales (221) - : +- BroadcastExchange (228) - : +- * Project (227) - : +- * Filter (226) - : +- * ColumnarToRow (225) - : +- Scan parquet default.date_dim (224) - +- * Project (236) - +- * BroadcastHashJoin Inner BuildRight (235) - :- * Filter (233) - : +- * ColumnarToRow (232) - : +- Scan parquet default.web_sales (231) - +- ReusedExchange (234) - - -(211) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (243) ++- Exchange (242) + +- * HashAggregate (241) + +- Union (240) + :- * Project (223) + : +- * BroadcastHashJoin Inner BuildRight (222) + : :- * Filter (216) + : : +- * ColumnarToRow (215) + : : +- Scan parquet default.store_sales (214) + : +- BroadcastExchange (221) + : +- * Project (220) + : +- * Filter (219) + : +- * ColumnarToRow (218) + : +- Scan parquet default.date_dim (217) + :- * Project (233) + : +- * BroadcastHashJoin Inner BuildRight (232) + : :- * Filter (226) + : : +- * ColumnarToRow (225) + : : +- Scan parquet default.catalog_sales (224) + : +- BroadcastExchange (231) + : +- * Project (230) + : +- * Filter (229) + : +- * ColumnarToRow (228) + : +- Scan parquet default.date_dim (227) + +- * Project (239) + +- * BroadcastHashJoin Inner BuildRight (238) + :- * Filter (236) + : +- * ColumnarToRow (235) + : +- Scan parquet default.web_sales (234) + +- ReusedExchange (237) + + +(214) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(212) ColumnarToRow [codegen id : 2] +(215) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(213) Filter [codegen id : 2] +(216) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(214) Scan parquet default.date_dim +(217) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(215) ColumnarToRow [codegen id : 1] +(218) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(216) Filter [codegen id : 1] +(219) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(217) Project [codegen id : 1] +(220) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(218) BroadcastExchange +(221) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#262] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#264] -(219) BroadcastHashJoin [codegen id : 2] +(222) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(220) Project [codegen id : 2] -Output [2]: [ss_quantity#3 AS quantity#263, ss_list_price#4 AS list_price#264] +(223) Project [codegen id : 2] +Output [2]: [ss_quantity#3 AS quantity#265, ss_list_price#4 AS list_price#266] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(221) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] +(224) Scan parquet default.catalog_sales +Output [3]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(222) ColumnarToRow [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] +(225) ColumnarToRow [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51] -(223) Filter [codegen id : 4] -Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] +(226) Filter [codegen id : 4] +Input [3]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51] Condition : isnotnull(cs_sold_date_sk#18) -(224) Scan parquet default.date_dim +(227) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(225) ColumnarToRow [codegen id : 3] +(228) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(226) Filter [codegen id : 3] +(229) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(227) Project [codegen id : 3] +(230) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(228) BroadcastExchange +(231) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#265] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#267] -(229) BroadcastHashJoin [codegen id : 4] +(232) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(230) Project [codegen id : 4] -Output [2]: [cs_quantity#48 AS quantity#266, cs_list_price#49 AS list_price#267] -Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] +(233) Project [codegen id : 4] +Output [2]: [cs_quantity#50 AS quantity#268, cs_list_price#51 AS list_price#269] +Input [4]: [cs_sold_date_sk#18, cs_quantity#50, cs_list_price#51, d_date_sk#10] -(231) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] +(234) Scan parquet default.web_sales +Output [3]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(232) ColumnarToRow [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] +(235) ColumnarToRow [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67] -(233) Filter [codegen id : 6] -Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -Condition : isnotnull(ws_sold_date_sk#22) +(236) Filter [codegen id : 6] +Input [3]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67] +Condition : isnotnull(ws_sold_date_sk#24) -(234) ReusedExchange [Reuses operator id: 228] +(237) ReusedExchange [Reuses operator id: 231] Output [1]: [d_date_sk#10] -(235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#22] +(238) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#24] Right keys [1]: [d_date_sk#10] Join condition: None -(236) Project [codegen id : 6] -Output [2]: [ws_quantity#64 AS quantity#268, ws_list_price#65 AS list_price#269] -Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] +(239) Project [codegen id : 6] +Output [2]: [ws_quantity#66 AS quantity#270, ws_list_price#67 AS list_price#271] +Input [4]: [ws_sold_date_sk#24, ws_quantity#66, ws_list_price#67, d_date_sk#10] -(237) Union +(240) Union -(238) HashAggregate [codegen id : 7] -Input [2]: [quantity#263, list_price#264] +(241) HashAggregate [codegen id : 7] +Input [2]: [quantity#265, list_price#266] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#270, count#271] -Results [2]: [sum#272, count#273] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#265 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#266 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#272, count#273] +Results [2]: [sum#274, count#275] -(239) Exchange -Input [2]: [sum#272, count#273] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#274] +(242) Exchange +Input [2]: [sum#274, count#275] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#276] -(240) HashAggregate [codegen id : 8] -Input [2]: [sum#272, count#273] +(243) HashAggregate [codegen id : 8] +Input [2]: [sum#274, count#275] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))#275] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))#275 AS average_sales#276] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#265 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#266 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#265 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#266 as decimal(12,2)))), DecimalType(18,2), true))#277] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#265 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#266 as decimal(12,2)))), DecimalType(18,2), true))#277 AS average_sales#278] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:4 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:5 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:6 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:7 Hosting operator id = 151 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:7 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:8 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:8 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:9 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:10 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:10 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:11 Hosting operator id = 174 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:11 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:12 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:12 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:13 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:13 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:14 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:14 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:15 Hosting operator id = 197 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:15 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index c63f1b8a75643..94a06475e4fc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (601) + WholeStageCodegen (691) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (600) + WholeStageCodegen (690) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (119) + WholeStageCodegen (137) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (118) + WholeStageCodegen (136) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (39) + WholeStageCodegen (45) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter - Exchange #17 + Exchange #19 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -35,7 +35,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter - BroadcastExchange #18 + BroadcastExchange #20 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter - BroadcastExchange #19 + BroadcastExchange #21 WholeStageCodegen (3) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -65,11 +65,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter - ReusedExchange [d_date_sk] #19 + ReusedExchange [d_date_sk] #21 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (38) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -86,11 +86,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] - WholeStageCodegen (18) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (17) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -99,116 +99,125 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (16) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Exchange [brand_id,class_id,category_id] #10 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #9 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [d_date_sk] #11 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #15 + BroadcastExchange #17 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (21) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (20) + Exchange [i_item_sk] #18 + WholeStageCodegen (23) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (37) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #5 - WholeStageCodegen (78) + WholeStageCodegen (90) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #20 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #22 + WholeStageCodegen (89) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -216,31 +225,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (41) + WholeStageCodegen (47) Sort [cs_item_sk] InputAdapter - Exchange [cs_item_sk] #21 - WholeStageCodegen (40) + Exchange [cs_item_sk] #23 + WholeStageCodegen (46) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] - WholeStageCodegen (57) + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #14 + ReusedExchange [d_date_sk] #16 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (117) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (135) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #22 - WholeStageCodegen (116) + Exchange [i_brand_id,i_class_id,i_category_id] #24 + WholeStageCodegen (134) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -248,156 +257,156 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (80) + WholeStageCodegen (92) Sort [ws_item_sk] InputAdapter - Exchange [ws_item_sk] #23 - WholeStageCodegen (79) + Exchange [ws_item_sk] #25 + WholeStageCodegen (91) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] - WholeStageCodegen (96) + WholeStageCodegen (111) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #14 + ReusedExchange [d_date_sk] #16 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (239) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (275) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #24 - WholeStageCodegen (238) + Exchange [channel,i_brand_id,i_class_id] #26 + WholeStageCodegen (274) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (237) + Exchange [channel,i_brand_id,i_class_id,i_category_id] #27 + WholeStageCodegen (273) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (158) + WholeStageCodegen (182) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #3 - WholeStageCodegen (197) + WholeStageCodegen (227) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 - WholeStageCodegen (236) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 + WholeStageCodegen (272) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 - WholeStageCodegen (359) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24 + WholeStageCodegen (413) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #26 - WholeStageCodegen (358) + Exchange [channel,i_brand_id] #28 + WholeStageCodegen (412) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #27 - WholeStageCodegen (357) + Exchange [channel,i_brand_id,i_class_id,i_category_id] #29 + WholeStageCodegen (411) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (278) + WholeStageCodegen (320) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #3 - WholeStageCodegen (317) + WholeStageCodegen (365) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 - WholeStageCodegen (356) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 + WholeStageCodegen (410) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 - WholeStageCodegen (479) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24 + WholeStageCodegen (551) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #28 - WholeStageCodegen (478) + Exchange [channel] #30 + WholeStageCodegen (550) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #29 - WholeStageCodegen (477) + Exchange [channel,i_brand_id,i_class_id,i_category_id] #31 + WholeStageCodegen (549) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (398) + WholeStageCodegen (458) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #3 - WholeStageCodegen (437) + WholeStageCodegen (503) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 - WholeStageCodegen (476) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 + WholeStageCodegen (548) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 - WholeStageCodegen (599) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24 + WholeStageCodegen (689) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #30 - WholeStageCodegen (598) + Exchange #32 + WholeStageCodegen (688) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #31 - WholeStageCodegen (597) + Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 + WholeStageCodegen (687) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (518) + WholeStageCodegen (596) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #3 - WholeStageCodegen (557) + WholeStageCodegen (641) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 - WholeStageCodegen (596) + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 + WholeStageCodegen (686) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24