diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 002ffcb29145..4b9ae9a7a81e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -238,7 +238,7 @@ abstract class Expression extends TreeNode[Expression] { * * See [[Canonicalize]] for more details. */ - def semanticEquals(other: Expression): Boolean = + final def semanticEquals(other: Expression): Boolean = deterministic && other.deterministic && canonicalized == other.canonicalized /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 52487d4decb6..2b8265f57ba7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -278,11 +278,6 @@ case class AttributeReference( case _ => false } - override def semanticEquals(other: Expression): Boolean = other match { - case ar: AttributeReference => sameRef(ar) - case _ => false - } - override def semanticHash(): Int = { this.exprId.hashCode() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 66cb26795de0..d1e953dc020a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -76,13 +76,6 @@ abstract class SubqueryExpression( AttributeSet.fromAttributeSets(outerAttrs.map(_.references)) override def children: Seq[Expression] = outerAttrs ++ joinCond override def withNewPlan(plan: LogicalPlan): SubqueryExpression - override def semanticEquals(o: Expression): Boolean = o match { - case p: SubqueryExpression => - this.getClass.getName.equals(p.getClass.getName) && plan.sameResult(p.plan) && - children.length == p.children.length && - children.zip(p.children).forall(p => p._1.semanticEquals(p._2)) - case _ => false - } } object SubqueryExpression { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index ac31a68b2b61..1805189b268d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -170,4 +170,11 @@ class CanonicalizeSuite extends SparkFunSuite { assert(nestedExpr2.canonicalized != nestedExpr3.canonicalized) } } + + test("SPARK-35742: Expression.semanticEquals should be symmetrical") { + val attr = AttributeReference("col", IntegerType)() + val expr = PromotePrecision(attr) + assert(expr.semanticEquals(attr)) + assert(attr.semanticEquals(expr)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryAdaptiveBroadcastExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryAdaptiveBroadcastExec.scala index a735d913c953..f0fa9a64a334 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryAdaptiveBroadcastExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SubqueryAdaptiveBroadcastExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan /** * Similar to [[SubqueryBroadcastExec]], this node is used to store the @@ -40,6 +41,11 @@ case class SubqueryAdaptiveBroadcastExec( "SubqueryAdaptiveBroadcastExec does not support the execute() code path.") } + protected override def doCanonicalize(): SparkPlan = { + val keys = buildKeys.map(k => QueryPlan.normalizeExpressions(k, child.output)) + copy(name = "dpp", buildKeys = keys, child = child.canonicalized) + } + override protected def withNewChildInternal(newChild: SparkPlan): SubqueryAdaptiveBroadcastExec = copy(child = newChild) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 60023af64649..eaa44561b8b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -73,7 +73,9 @@ case class HashAggregateExec( // This is for testing. We force TungstenAggregationIterator to fall back to the unsafe row hash // map and/or the sort-based aggregation once it has processed a given number of input rows. private val testFallbackStartsAt: Option[(Int, Int)] = { - sqlContext.getConf("spark.sql.TungstenAggregate.testFallbackStartsAt", null) match { + Option(sqlContext).map { sc => + sc.getConf("spark.sql.TungstenAggregate.testFallbackStartsAt", null) + }.orNull match { case null | "" => None case fallbackStartsAt => val splits = fallbackStartsAt.split(",").map(_.trim) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 06f5d2c3f070..cc141e9cfe98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -71,9 +71,8 @@ case class ScalarSubquery( override def toString: String = plan.simpleString(SQLConf.get.maxToStringFields) override def withNewPlan(query: BaseSubqueryExec): ScalarSubquery = copy(plan = query) - override def semanticEquals(other: Expression): Boolean = other match { - case s: ScalarSubquery => plan.sameResult(s.plan) - case _ => false + override lazy val canonicalized: Expression = { + ScalarSubquery(plan.canonicalized.asInstanceOf[BaseSubqueryExec], ExprId(0)) } // the first column in first row from `query`. @@ -127,11 +126,6 @@ case class InSubqueryExec( override def withNewPlan(plan: BaseSubqueryExec): InSubqueryExec = copy(plan = plan) final override def nodePatternsInternal: Seq[TreePattern] = Seq(IN_SUBQUERY_EXEC) - override def semanticEquals(other: Expression): Boolean = other match { - case in: InSubqueryExec => child.semanticEquals(in.child) && plan.sameResult(in.plan) - case _ => false - } - def updateResult(): Unit = { val rows = plan.executeCollect() result = if (plan.output.length > 1) { diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index a7d797dbb545..2eb4beb63497 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -TakeOrderedAndProject (103) -+- Union (102) +TakeOrderedAndProject (95) ++- Union (94) :- * HashAggregate (70) : +- Exchange (69) : +- * HashAggregate (68) @@ -71,13 +71,13 @@ TakeOrderedAndProject (103) : +- * Filter (63) : +- * ColumnarToRow (62) : +- Scan parquet default.date_dim (61) - +- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Project (98) - +- * BroadcastHashJoin Inner BuildRight (97) - :- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * Project (90) + +- * BroadcastHashJoin Inner BuildRight (89) + :- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) : :- * SortMergeJoin LeftSemi (84) : : :- * Sort (78) : : : +- Exchange (77) @@ -92,16 +92,8 @@ TakeOrderedAndProject (103) : : +- * Filter (81) : : +- * HashAggregate (80) : : +- ReusedExchange (79) - : +- BroadcastExchange (93) - : +- * SortMergeJoin LeftSemi (92) - : :- * Sort (86) - : : +- ReusedExchange (85) - : +- * Sort (91) - : +- * Project (90) - : +- * Filter (89) - : +- * HashAggregate (88) - : +- ReusedExchange (87) - +- ReusedExchange (96) + : +- ReusedExchange (85) + +- ReusedExchange (88) (1) Scan parquet default.catalog_sales @@ -492,245 +484,210 @@ Left keys [1]: [ws_bill_customer_sk#60] Right keys [1]: [c_customer_sk#65] Join condition: None -(85) ReusedExchange [Reuses operator id: 50] +(85) ReusedExchange [Reuses operator id: unknown] Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -(86) Sort [codegen id : 29] -Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 - -(87) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] - -(88) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] - -(89) Filter [codegen id : 32] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) - -(90) Project [codegen id : 32] -Output [1]: [c_customer_sk#65] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] - -(91) Sort [codegen id : 32] -Input [1]: [c_customer_sk#65] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 - -(92) SortMergeJoin [codegen id : 33] -Left keys [1]: [c_customer_sk#72] -Right keys [1]: [c_customer_sk#65] -Join condition: None - -(93) BroadcastExchange -Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] - -(94) BroadcastHashJoin [codegen id : 35] +(86) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#60] Right keys [1]: [c_customer_sk#72] Join condition: None -(95) Project [codegen id : 35] +(87) Project [codegen id : 35] Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] -(96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#76] +(88) ReusedExchange [Reuses operator id: 65] +Output [1]: [d_date_sk#75] -(97) BroadcastHashJoin [codegen id : 35] +(89) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#76] +Right keys [1]: [d_date_sk#75] Join condition: None -(98) Project [codegen id : 35] +(90) Project [codegen id : 35] Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] +Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#75] -(99) HashAggregate [codegen id : 35] +(91) HashAggregate [codegen id : 35] Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] Keys [2]: [c_last_name#74, c_first_name#73] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#77, isEmpty#78] -Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79] -(100) Exchange -Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] +(92) Exchange +Input [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79] +Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#80] -(101) HashAggregate [codegen id : 36] -Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +(93) HashAggregate [codegen id : 36] +Input [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79] Keys [2]: [c_last_name#74, c_first_name#73] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#81] +Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#82] -(102) Union +(94) Union -(103) TakeOrderedAndProject +(95) TakeOrderedAndProject Input [3]: [c_last_name#45, c_first_name#44, sales#58] Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (104) +ReusedExchange (96) -(104) ReusedExchange [Reuses operator id: 65] +(96) ReusedExchange [Reuses operator id: 65] Output [1]: [d_date_sk#48] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (105) +ReusedExchange (97) -(105) ReusedExchange [Reuses operator id: 11] +(97) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] -* HashAggregate (127) -+- Exchange (126) - +- * HashAggregate (125) - +- * HashAggregate (124) - +- Exchange (123) - +- * HashAggregate (122) - +- * Project (121) - +- * BroadcastHashJoin Inner BuildRight (120) - :- * Project (114) - : +- * BroadcastHashJoin Inner BuildRight (113) - : :- * Filter (108) - : : +- * ColumnarToRow (107) - : : +- Scan parquet default.store_sales (106) - : +- BroadcastExchange (112) - : +- * Filter (111) - : +- * ColumnarToRow (110) - : +- Scan parquet default.customer (109) - +- BroadcastExchange (119) - +- * Project (118) - +- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet default.date_dim (115) - - -(106) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +* HashAggregate (119) ++- Exchange (118) + +- * HashAggregate (117) + +- * HashAggregate (116) + +- Exchange (115) + +- * HashAggregate (114) + +- * Project (113) + +- * BroadcastHashJoin Inner BuildRight (112) + :- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Filter (100) + : : +- * ColumnarToRow (99) + : : +- Scan parquet default.store_sales (98) + : +- BroadcastExchange (104) + : +- * Filter (103) + : +- * ColumnarToRow (102) + : +- Scan parquet default.customer (101) + +- BroadcastExchange (111) + +- * Project (110) + +- * Filter (109) + +- * ColumnarToRow (108) + +- Scan parquet default.date_dim (107) + + +(98) Scan parquet default.store_sales +Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +(99) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] -(108) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_customer_sk#84) +(100) Filter [codegen id : 3] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Condition : isnotnull(ss_customer_sk#83) -(109) Scan parquet default.customer -Output [1]: [c_customer_sk#89] +(101) Scan parquet default.customer +Output [1]: [c_customer_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(110) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#89] +(102) ColumnarToRow [codegen id : 1] +Input [1]: [c_customer_sk#88] -(111) Filter [codegen id : 1] -Input [1]: [c_customer_sk#89] -Condition : isnotnull(c_customer_sk#89) +(103) Filter [codegen id : 1] +Input [1]: [c_customer_sk#88] +Condition : isnotnull(c_customer_sk#88) -(112) BroadcastExchange -Input [1]: [c_customer_sk#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] +(104) BroadcastExchange +Input [1]: [c_customer_sk#88] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#89] -(113) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#84] -Right keys [1]: [c_customer_sk#89] +(105) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#83] +Right keys [1]: [c_customer_sk#88] Join condition: None -(114) Project [codegen id : 3] -Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] -Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] +(106) Project [codegen id : 3] +Output [4]: [ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88] +Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88] -(115) Scan parquet default.date_dim -Output [2]: [d_date_sk#91, d_year#92] +(107) Scan parquet default.date_dim +Output [2]: [d_date_sk#90, d_year#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#91, d_year#92] +(108) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#90, d_year#91] -(117) Filter [codegen id : 2] -Input [2]: [d_date_sk#91, d_year#92] -Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91)) +(109) Filter [codegen id : 2] +Input [2]: [d_date_sk#90, d_year#91] +Condition : (d_year#91 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#90)) -(118) Project [codegen id : 2] -Output [1]: [d_date_sk#91] -Input [2]: [d_date_sk#91, d_year#92] +(110) Project [codegen id : 2] +Output [1]: [d_date_sk#90] +Input [2]: [d_date_sk#90, d_year#91] -(119) BroadcastExchange -Input [1]: [d_date_sk#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +(111) BroadcastExchange +Input [1]: [d_date_sk#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] -(120) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#91] +(112) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#90] Join condition: None -(121) Project [codegen id : 3] -Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] -Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91] - -(122) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] -Keys [1]: [c_customer_sk#89] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#94, isEmpty#95] -Results [3]: [c_customer_sk#89, sum#96, isEmpty#97] - -(123) Exchange -Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] -Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98] - -(124) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] -Keys [1]: [c_customer_sk#89] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100] - -(125) HashAggregate [codegen id : 4] -Input [1]: [csales#100] +(113) Project [codegen id : 3] +Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#88] +Input [5]: [ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88, d_date_sk#90] + +(114) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#88] +Keys [1]: [c_customer_sk#88] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#93, isEmpty#94] +Results [3]: [c_customer_sk#88, sum#95, isEmpty#96] + +(115) Exchange +Input [3]: [c_customer_sk#88, sum#95, isEmpty#96] +Arguments: hashpartitioning(c_customer_sk#88, 5), ENSURE_REQUIREMENTS, [id=#97] + +(116) HashAggregate [codegen id : 4] +Input [3]: [c_customer_sk#88, sum#95, isEmpty#96] +Keys [1]: [c_customer_sk#88] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] + +(117) HashAggregate [codegen id : 4] +Input [1]: [csales#99] Keys: [] -Functions [1]: [partial_max(csales#100)] -Aggregate Attributes [1]: [max#101] -Results [1]: [max#102] +Functions [1]: [partial_max(csales#99)] +Aggregate Attributes [1]: [max#100] +Results [1]: [max#101] -(126) Exchange -Input [1]: [max#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +(118) Exchange +Input [1]: [max#101] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] -(127) HashAggregate [codegen id : 5] -Input [1]: [max#102] +(119) HashAggregate [codegen id : 5] +Input [1]: [max#101] Keys: [] -Functions [1]: [max(csales#100)] -Aggregate Attributes [1]: [max(csales#100)#104] -Results [1]: [max(csales#100)#104 AS tpcds_cmax#105] +Functions [1]: [max(csales#99)] +Aggregate Attributes [1]: [max(csales#99)#103] +Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] -Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88 -ReusedExchange (128) +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 +ReusedExchange (120) -(128) ReusedExchange [Reuses operator id: 119] -Output [1]: [d_date_sk#91] +(120) ReusedExchange [Reuses operator id: 111] +Output [1]: [d_date_sk#90] Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] @@ -738,6 +695,4 @@ Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN d Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] - diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 8ef321be35bf..7127bcfcc2b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -188,22 +188,6 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter - BroadcastExchange #19 - WholeStageCodegen (33) - SortMergeJoin [c_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - InputAdapter - WholeStageCodegen (32) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #19 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index 84a8547e3f6e..d75c9039fd21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) +TakeOrderedAndProject (32) ++- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (14) : : : +- * Filter (13) : : : +- Window (12) @@ -20,21 +20,17 @@ TakeOrderedAndProject (36) : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- Window (21) - : : +- * Sort (20) - : : +- Exchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * HashAggregate (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.item (27) - +- ReusedExchange (33) + : : +- BroadcastExchange (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- Window (17) + : : +- * Sort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.item (23) + +- ReusedExchange (29) (1) Scan parquet default.store_sales @@ -101,148 +97,127 @@ Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) Output [2]: [item_sk#11, rnk#17] Input [3]: [item_sk#11, rank_col#12, rnk#17] -(15) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#18, sum#19, count#20] +(15) ReusedExchange [Reuses operator id: 10] +Output [2]: [item_sk#18, rank_col#19] -(16) HashAggregate [codegen id : 5] -Input [3]: [ss_item_sk#18, sum#19, count#20] -Keys [1]: [ss_item_sk#18] -Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] -Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] +(16) Sort [codegen id : 6] +Input [2]: [item_sk#18, rank_col#19] +Arguments: [rank_col#19 DESC NULLS LAST], false, 0 -(17) Filter [codegen id : 5] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] -Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +(17) Window +Input [2]: [item_sk#18, rank_col#19] +Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST] -(18) Project [codegen id : 5] -Output [2]: [item_sk#23, rank_col#24] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +(18) Filter [codegen id : 7] +Input [3]: [item_sk#18, rank_col#19, rnk#20] +Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18)) -(19) Exchange -Input [2]: [item_sk#23, rank_col#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +(19) Project [codegen id : 7] +Output [2]: [item_sk#18, rnk#20] +Input [3]: [item_sk#18, rank_col#19, rnk#20] -(20) Sort [codegen id : 6] -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank_col#24 DESC NULLS LAST], false, 0 +(20) BroadcastExchange +Input [2]: [item_sk#18, rnk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#21] -(21) Window -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] - -(22) Filter [codegen id : 7] -Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) - -(23) Project [codegen id : 7] -Output [2]: [item_sk#23, rnk#27] -Input [3]: [item_sk#23, rank_col#24, rnk#27] - -(24) BroadcastExchange -Input [2]: [item_sk#23, rnk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28] - -(25) BroadcastHashJoin [codegen id : 10] +(21) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#27] +Right keys [1]: [rnk#20] Join condition: None -(26) Project [codegen id : 10] -Output [3]: [item_sk#11, rnk#17, item_sk#23] -Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] +(22) Project [codegen id : 10] +Output [3]: [item_sk#11, rnk#17, item_sk#18] +Input [4]: [item_sk#11, rnk#17, item_sk#18, rnk#20] -(27) Scan parquet default.item -Output [2]: [i_item_sk#29, i_product_name#30] +(23) Scan parquet default.item +Output [2]: [i_item_sk#22, i_product_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#29, i_product_name#30] +(24) ColumnarToRow [codegen id : 8] +Input [2]: [i_item_sk#22, i_product_name#23] -(29) Filter [codegen id : 8] -Input [2]: [i_item_sk#29, i_product_name#30] -Condition : isnotnull(i_item_sk#29) +(25) Filter [codegen id : 8] +Input [2]: [i_item_sk#22, i_product_name#23] +Condition : isnotnull(i_item_sk#22) -(30) BroadcastExchange -Input [2]: [i_item_sk#29, i_product_name#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +(26) BroadcastExchange +Input [2]: [i_item_sk#22, i_product_name#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] -(31) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#29] +Right keys [1]: [i_item_sk#22] Join condition: None -(32) Project [codegen id : 10] -Output [3]: [rnk#17, item_sk#23, i_product_name#30] -Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#29, i_product_name#30] +(28) Project [codegen id : 10] +Output [3]: [rnk#17, item_sk#18, i_product_name#23] +Input [5]: [item_sk#11, rnk#17, item_sk#18, i_item_sk#22, i_product_name#23] -(33) ReusedExchange [Reuses operator id: 30] -Output [2]: [i_item_sk#32, i_product_name#33] +(29) ReusedExchange [Reuses operator id: 26] +Output [2]: [i_item_sk#25, i_product_name#26] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#23] -Right keys [1]: [i_item_sk#32] +(30) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [item_sk#18] +Right keys [1]: [i_item_sk#25] Join condition: None -(35) Project [codegen id : 10] -Output [3]: [rnk#17, i_product_name#30 AS best_performing#34, i_product_name#33 AS worst_performing#35] -Input [5]: [rnk#17, item_sk#23, i_product_name#30, i_item_sk#32, i_product_name#33] +(31) Project [codegen id : 10] +Output [3]: [rnk#17, i_product_name#23 AS best_performing#27, i_product_name#26 AS worst_performing#28] +Input [5]: [rnk#17, item_sk#18, i_product_name#23, i_item_sk#25, i_product_name#26] -(36) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#34, worst_performing#35] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#34, worst_performing#35] +(32) TakeOrderedAndProject +Input [3]: [rnk#17, best_performing#27, worst_performing#28] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#27, worst_performing#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* HashAggregate (43) -+- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * Filter (39) - +- * ColumnarToRow (38) - +- Scan parquet default.store_sales (37) +* HashAggregate (39) ++- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * Filter (35) + +- * ColumnarToRow (34) + +- Scan parquet default.store_sales (33) -(37) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +(33) Scan parquet default.store_sales +Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] - -(39) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] -Condition : ((isnotnull(ss_store_sk#37) AND (ss_store_sk#37 = 4)) AND isnull(ss_addr_sk#36)) - -(40) Project [codegen id : 1] -Output [2]: [ss_store_sk#37, ss_net_profit#38] -Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] - -(41) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#37, ss_net_profit#38] -Keys [1]: [ss_store_sk#37] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#38))] -Aggregate Attributes [2]: [sum#40, count#41] -Results [3]: [ss_store_sk#37, sum#42, count#43] - -(42) Exchange -Input [3]: [ss_store_sk#37, sum#42, count#43] -Arguments: hashpartitioning(ss_store_sk#37, 5), ENSURE_REQUIREMENTS, [id=#44] - -(43) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#37, sum#42, count#43] -Keys [1]: [ss_store_sk#37] -Functions [1]: [avg(UnscaledValue(ss_net_profit#38))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#38))#45] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#38))#45 / 100.0) as decimal(11,6)) AS rank_col#46] - -Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] +(34) ColumnarToRow [codegen id : 1] +Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] + +(35) Filter [codegen id : 1] +Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] +Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29)) + +(36) Project [codegen id : 1] +Output [2]: [ss_store_sk#30, ss_net_profit#31] +Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] + +(37) HashAggregate [codegen id : 1] +Input [2]: [ss_store_sk#30, ss_net_profit#31] +Keys [1]: [ss_store_sk#30] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#30, sum#35, count#36] + +(38) Exchange +Input [3]: [ss_store_sk#30, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [id=#37] + +(39) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#30, sum#35, count#36] +Keys [1]: [ss_store_sk#30] +Functions [1]: [avg(UnscaledValue(ss_net_profit#31))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#38] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#38 / 100.0) as decimal(11,6)) AS rank_col#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt index 73bc3a333b63..ffc04edf3dfb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt @@ -49,20 +49,13 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (6) Sort [rank_col] InputAdapter - Exchange #5 - WholeStageCodegen (5) - Project [item_sk,rank_col] - Filter [avg(ss_net_profit)] - ReusedSubquery [rank_col] #1 - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] - InputAdapter - ReusedExchange [ss_item_sk,sum,count] #2 + ReusedExchange [item_sk,rank_col] #1 InputAdapter - BroadcastExchange #6 + BroadcastExchange #5 WholeStageCodegen (8) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_product_name] InputAdapter - ReusedExchange [i_item_sk,i_product_name] #6 + ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index b42ddd8db783..6bb069211533 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * SortMergeJoin Inner (26) +TakeOrderedAndProject (33) ++- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (23) + : : +- * SortMergeJoin Inner (22) : : :- * Sort (15) : : : +- * Project (14) : : : +- * Filter (13) @@ -21,21 +21,17 @@ TakeOrderedAndProject (37) : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet default.store_sales (1) - : : +- * Sort (25) - : : +- * Project (24) - : : +- * Filter (23) - : : +- Window (22) - : : +- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- * HashAggregate (17) - : : +- ReusedExchange (16) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.item (28) - +- ReusedExchange (34) + : : +- * Sort (21) + : : +- * Project (20) + : : +- * Filter (19) + : : +- Window (18) + : : +- * Sort (17) + : : +- ReusedExchange (16) + : +- BroadcastExchange (27) + : +- * Filter (26) + : +- * ColumnarToRow (25) + : +- Scan parquet default.item (24) + +- ReusedExchange (30) (1) Scan parquet default.store_sales @@ -106,148 +102,127 @@ Input [3]: [item_sk#11, rank_col#12, rnk#17] Input [2]: [item_sk#11, rnk#17] Arguments: [rnk#17 ASC NULLS FIRST], false, 0 -(16) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#18, sum#19, count#20] +(16) ReusedExchange [Reuses operator id: 10] +Output [2]: [item_sk#18, rank_col#19] -(17) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#18, sum#19, count#20] -Keys [1]: [ss_item_sk#18] -Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] -Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] +(17) Sort [codegen id : 7] +Input [2]: [item_sk#18, rank_col#19] +Arguments: [rank_col#19 DESC NULLS LAST], false, 0 -(18) Filter [codegen id : 6] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] -Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +(18) Window +Input [2]: [item_sk#18, rank_col#19] +Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST] -(19) Project [codegen id : 6] -Output [2]: [item_sk#23, rank_col#24] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +(19) Filter [codegen id : 8] +Input [3]: [item_sk#18, rank_col#19, rnk#20] +Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18)) -(20) Exchange -Input [2]: [item_sk#23, rank_col#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +(20) Project [codegen id : 8] +Output [2]: [item_sk#18, rnk#20] +Input [3]: [item_sk#18, rank_col#19, rnk#20] -(21) Sort [codegen id : 7] -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank_col#24 DESC NULLS LAST], false, 0 +(21) Sort [codegen id : 8] +Input [2]: [item_sk#18, rnk#20] +Arguments: [rnk#20 ASC NULLS FIRST], false, 0 -(22) Window -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] - -(23) Filter [codegen id : 8] -Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) - -(24) Project [codegen id : 8] -Output [2]: [item_sk#23, rnk#27] -Input [3]: [item_sk#23, rank_col#24, rnk#27] - -(25) Sort [codegen id : 8] -Input [2]: [item_sk#23, rnk#27] -Arguments: [rnk#27 ASC NULLS FIRST], false, 0 - -(26) SortMergeJoin [codegen id : 11] +(22) SortMergeJoin [codegen id : 11] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#27] +Right keys [1]: [rnk#20] Join condition: None -(27) Project [codegen id : 11] -Output [3]: [item_sk#11, rnk#17, item_sk#23] -Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] +(23) Project [codegen id : 11] +Output [3]: [item_sk#11, rnk#17, item_sk#18] +Input [4]: [item_sk#11, rnk#17, item_sk#18, rnk#20] -(28) Scan parquet default.item -Output [2]: [i_item_sk#28, i_product_name#29] +(24) Scan parquet default.item +Output [2]: [i_item_sk#21, i_product_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#28, i_product_name#29] +(25) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_sk#21, i_product_name#22] -(30) Filter [codegen id : 9] -Input [2]: [i_item_sk#28, i_product_name#29] -Condition : isnotnull(i_item_sk#28) +(26) Filter [codegen id : 9] +Input [2]: [i_item_sk#21, i_product_name#22] +Condition : isnotnull(i_item_sk#21) -(31) BroadcastExchange -Input [2]: [i_item_sk#28, i_product_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +(27) BroadcastExchange +Input [2]: [i_item_sk#21, i_product_name#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(32) BroadcastHashJoin [codegen id : 11] +(28) BroadcastHashJoin [codegen id : 11] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#28] +Right keys [1]: [i_item_sk#21] Join condition: None -(33) Project [codegen id : 11] -Output [3]: [rnk#17, item_sk#23, i_product_name#29] -Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#28, i_product_name#29] +(29) Project [codegen id : 11] +Output [3]: [rnk#17, item_sk#18, i_product_name#22] +Input [5]: [item_sk#11, rnk#17, item_sk#18, i_item_sk#21, i_product_name#22] -(34) ReusedExchange [Reuses operator id: 31] -Output [2]: [i_item_sk#31, i_product_name#32] +(30) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#24, i_product_name#25] -(35) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#23] -Right keys [1]: [i_item_sk#31] +(31) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [item_sk#18] +Right keys [1]: [i_item_sk#24] Join condition: None -(36) Project [codegen id : 11] -Output [3]: [rnk#17, i_product_name#29 AS best_performing#33, i_product_name#32 AS worst_performing#34] -Input [5]: [rnk#17, item_sk#23, i_product_name#29, i_item_sk#31, i_product_name#32] +(32) Project [codegen id : 11] +Output [3]: [rnk#17, i_product_name#22 AS best_performing#26, i_product_name#25 AS worst_performing#27] +Input [5]: [rnk#17, item_sk#18, i_product_name#22, i_item_sk#24, i_product_name#25] -(37) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#33, worst_performing#34] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#33, worst_performing#34] +(33) TakeOrderedAndProject +Input [3]: [rnk#17, best_performing#26, worst_performing#27] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#26, worst_performing#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* HashAggregate (44) -+- Exchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.store_sales (38) +* HashAggregate (40) ++- Exchange (39) + +- * HashAggregate (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.store_sales (34) -(38) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +(34) Scan parquet default.store_sales +Output [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] - -(40) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_store_sk#36) AND (ss_store_sk#36 = 4)) AND isnull(ss_addr_sk#35)) - -(41) Project [codegen id : 1] -Output [2]: [ss_store_sk#36, ss_net_profit#37] -Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] - -(42) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#36, ss_net_profit#37] -Keys [1]: [ss_store_sk#36] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#37))] -Aggregate Attributes [2]: [sum#39, count#40] -Results [3]: [ss_store_sk#36, sum#41, count#42] - -(43) Exchange -Input [3]: [ss_store_sk#36, sum#41, count#42] -Arguments: hashpartitioning(ss_store_sk#36, 5), ENSURE_REQUIREMENTS, [id=#43] - -(44) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#36, sum#41, count#42] -Keys [1]: [ss_store_sk#36] -Functions [1]: [avg(UnscaledValue(ss_net_profit#37))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#37))#44] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#37))#44 / 100.0) as decimal(11,6)) AS rank_col#45] - -Subquery:2 Hosting operator id = 18 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] +(35) ColumnarToRow [codegen id : 1] +Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] + +(36) Filter [codegen id : 1] +Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_store_sk#29) AND (ss_store_sk#29 = 4)) AND isnull(ss_addr_sk#28)) + +(37) Project [codegen id : 1] +Output [2]: [ss_store_sk#29, ss_net_profit#30] +Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] + +(38) HashAggregate [codegen id : 1] +Input [2]: [ss_store_sk#29, ss_net_profit#30] +Keys [1]: [ss_store_sk#29] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#30))] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#29, sum#34, count#35] + +(39) Exchange +Input [3]: [ss_store_sk#29, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#29, 5), ENSURE_REQUIREMENTS, [id=#36] + +(40) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#29, sum#34, count#35] +Keys [1]: [ss_store_sk#29] +Functions [1]: [avg(UnscaledValue(ss_net_profit#30))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#30))#37] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#30))#37 / 100.0) as decimal(11,6)) AS rank_col#38] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 10d3570c2dc8..f3642f6d949b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -52,20 +52,13 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (7) Sort [rank_col] InputAdapter - Exchange #4 - WholeStageCodegen (6) - Project [item_sk,rank_col] - Filter [avg(ss_net_profit)] - ReusedSubquery [rank_col] #1 - HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] - InputAdapter - ReusedExchange [ss_item_sk,sum,count] #2 + ReusedExchange [item_sk,rank_col] #1 InputAdapter - BroadcastExchange #5 + BroadcastExchange #4 WholeStageCodegen (9) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_product_name] InputAdapter - ReusedExchange [i_item_sk,i_product_name] #5 + ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index b339df0707d2..d6b61f63b41a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) +TakeOrderedAndProject (59) ++- * Project (58) + +- * BroadcastHashJoin Inner BuildRight (57) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) : :- * Filter (26) : : +- * HashAggregate (25) : : +- Exchange (24) @@ -30,54 +30,34 @@ TakeOrderedAndProject (79) : : +- * Filter (19) : : +- * ColumnarToRow (18) : : +- Scan parquet default.item (17) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) + : +- BroadcastExchange (40) + : +- * Filter (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) : : :- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet default.catalog_sales (27) - : : +- BroadcastExchange (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin LeftSemi BuildRight (38) - : : :- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet default.date_dim (30) - : : +- BroadcastExchange (37) - : : +- * Project (36) - : : +- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet default.date_dim (33) - : +- ReusedExchange (43) - +- BroadcastExchange (76) - +- * Filter (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet default.web_sales (53) - : +- BroadcastExchange (66) - : +- * Project (65) - : +- * BroadcastHashJoin LeftSemi BuildRight (64) - : :- * Filter (58) - : : +- * ColumnarToRow (57) - : : +- Scan parquet default.date_dim (56) - : +- BroadcastExchange (63) - : +- * Project (62) - : +- * Filter (61) - : +- * ColumnarToRow (60) - : +- Scan parquet default.date_dim (59) - +- ReusedExchange (69) + : : +- ReusedExchange (30) + : +- ReusedExchange (33) + +- BroadcastExchange (56) + +- * Filter (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Filter (45) + : : +- * ColumnarToRow (44) + : : +- Scan parquet default.web_sales (43) + : +- ReusedExchange (46) + +- ReusedExchange (49) (1) Scan parquet default.store_sales @@ -217,264 +197,168 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) -(30) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_date#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#24, d_date#25] - -(32) Filter [codegen id : 6] -Input [2]: [d_date_sk#24, d_date#25] -Condition : isnotnull(d_date_sk#24) - -(33) Scan parquet default.date_dim -Output [2]: [d_date#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#26, d_week_seq#27] - -(35) Filter [codegen id : 5] -Input [2]: [d_date#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND (d_week_seq#27 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) - -(36) Project [codegen id : 5] -Output [1]: [d_date#26] -Input [2]: [d_date#26, d_week_seq#27] - -(37) BroadcastExchange -Input [1]: [d_date#26] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#28] - -(38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#25] -Right keys [1]: [d_date#26] -Join condition: None - -(39) Project [codegen id : 6] +(30) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#25] -(40) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(41) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join condition: None -(42) Project [codegen id : 8] +(32) Project [codegen id : 8] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] -(43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#30, i_item_id#31] +(33) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#25, i_item_id#26] -(44) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#25] Join condition: None -(45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#31] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#30, i_item_id#31] +(35) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#22, i_item_id#26] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#25, i_item_id#26] -(46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#31] -Keys [1]: [i_item_id#31] +(36) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#22, i_item_id#26] +Keys [1]: [i_item_id#26] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#26, sum#28] -(47) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(37) Exchange +Input [2]: [i_item_id#26, sum#28] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#29] -(48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] +(38) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#26, sum#28] +Keys [1]: [i_item_id#26] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] -Results [2]: [i_item_id#31 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#26 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] -(49) Filter [codegen id : 9] -Input [2]: [item_id#36, cs_item_rev#37] -Condition : isnotnull(cs_item_rev#37) +(39) Filter [codegen id : 9] +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) -(50) BroadcastExchange -Input [2]: [item_id#36, cs_item_rev#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] +(40) BroadcastExchange +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] -(51) BroadcastHashJoin [codegen id : 15] +(41) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#36] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) -(52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] -Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] +(42) Project [codegen id : 15] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] -(53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +(43) Scan parquet default.web_sales +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +(44) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_item_sk#39) +(45) Filter [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) -(56) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_date#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#42, d_date#43] - -(58) Filter [codegen id : 11] -Input [2]: [d_date_sk#42, d_date#43] -Condition : isnotnull(d_date_sk#42) - -(59) Scan parquet default.date_dim -Output [2]: [d_date#44, d_week_seq#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct +(46) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#37] -(60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#44, d_week_seq#45] - -(61) Filter [codegen id : 10] -Input [2]: [d_date#44, d_week_seq#45] -Condition : (isnotnull(d_week_seq#45) AND (d_week_seq#45 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) - -(62) Project [codegen id : 10] -Output [1]: [d_date#44] -Input [2]: [d_date#44, d_week_seq#45] - -(63) BroadcastExchange -Input [1]: [d_date#44] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#46] - -(64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#43] -Right keys [1]: [d_date#44] -Join condition: None - -(65) Project [codegen id : 11] -Output [1]: [d_date_sk#42] -Input [2]: [d_date_sk#42, d_date#43] - -(66) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] - -(67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#41] -Right keys [1]: [d_date_sk#42] +(47) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None -(68) Project [codegen id : 13] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#40] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, d_date_sk#42] +(48) Project [codegen id : 13] +Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#37] -(69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#48, i_item_id#49] +(49) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#38, i_item_id#39] -(70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#48] +(50) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#38] Join condition: None -(71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#40, i_item_id#49] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, i_item_sk#48, i_item_id#49] - -(72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#40, i_item_id#49] -Keys [1]: [i_item_id#49] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum#50] -Results [2]: [i_item_id#49, sum#51] - -(73) Exchange -Input [2]: [i_item_id#49, sum#51] -Arguments: hashpartitioning(i_item_id#49, 5), ENSURE_REQUIREMENTS, [id=#52] - -(74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#49, sum#51] -Keys [1]: [i_item_id#49] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] -Results [2]: [i_item_id#49 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] - -(75) Filter [codegen id : 14] -Input [2]: [item_id#54, ws_item_rev#55] -Condition : isnotnull(ws_item_rev#55) - -(76) BroadcastExchange -Input [2]: [item_id#54, ws_item_rev#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] - -(77) BroadcastHashJoin [codegen id : 15] +(51) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#35, i_item_id#39] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#38, i_item_id#39] + +(52) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#35, i_item_id#39] +Keys [1]: [i_item_id#39] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#39, sum#41] + +(53) Exchange +Input [2]: [i_item_id#39, sum#41] +Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#42] + +(54) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#39, sum#41] +Keys [1]: [i_item_id#39] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#39 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] + +(55) Filter [codegen id : 14] +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) + +(56) BroadcastExchange +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] + +(57) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#54] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) -(78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] +(58) Project [codegen id : 15] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] -(79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +(59) TakeOrderedAndProject +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet default.date_dim (80) +* Project (63) ++- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet default.date_dim (60) -(80) Scan parquet default.date_dim -Output [2]: [d_date#61, d_week_seq#62] +(60) Scan parquet default.date_dim +Output [2]: [d_date#51, d_week_seq#52] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] - -(82) Filter [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] -Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) - -(83) Project [codegen id : 1] -Output [1]: [d_week_seq#62] -Input [2]: [d_date#61, d_week_seq#62] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] -Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +(62) Filter [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] +Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) -Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +(63) Project [codegen id : 1] +Output [1]: [d_week_seq#52] +Input [2]: [d_date#51, d_week_seq#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 5642c7dcbf91..57c79b9aee84 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -67,32 +67,16 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [d_date] - Filter [d_week_seq] - ReusedSubquery [d_week_seq] #1 - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -104,22 +88,6 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (11) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (10) - Project [d_date] - Filter [d_week_seq] - ReusedSubquery [d_week_seq] #1 - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 1f3e6853a3c4..a8ce33ee99d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) +TakeOrderedAndProject (59) ++- * Project (58) + +- * BroadcastHashJoin Inner BuildRight (57) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) : :- * Filter (26) : : +- * HashAggregate (25) : : +- Exchange (24) @@ -30,54 +30,34 @@ TakeOrderedAndProject (79) : : +- * Filter (15) : : +- * ColumnarToRow (14) : : +- Scan parquet default.date_dim (13) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) + : +- BroadcastExchange (40) + : +- * Filter (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) : :- * Project (32) : : +- * BroadcastHashJoin Inner BuildRight (31) : : :- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet default.catalog_sales (27) : : +- ReusedExchange (30) - : +- BroadcastExchange (43) - : +- * Project (42) - : +- * BroadcastHashJoin LeftSemi BuildRight (41) - : :- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet default.date_dim (33) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.date_dim (36) - +- BroadcastExchange (76) - +- * Filter (75) - +- * HashAggregate (74) - +- Exchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet default.web_sales (53) - : +- ReusedExchange (56) - +- BroadcastExchange (69) - +- * Project (68) - +- * BroadcastHashJoin LeftSemi BuildRight (67) - :- * Filter (61) - : +- * ColumnarToRow (60) - : +- Scan parquet default.date_dim (59) - +- BroadcastExchange (66) - +- * Project (65) - +- * Filter (64) - +- * ColumnarToRow (63) - +- Scan parquet default.date_dim (62) + : +- ReusedExchange (33) + +- BroadcastExchange (56) + +- * Filter (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * BroadcastHashJoin Inner BuildRight (50) + :- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Filter (45) + : : +- * ColumnarToRow (44) + : : +- Scan parquet default.web_sales (43) + : +- ReusedExchange (46) + +- ReusedExchange (49) (1) Scan parquet default.store_sales @@ -229,252 +209,156 @@ Join condition: None Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] -(33) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_date#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#26, d_date#27] - -(35) Filter [codegen id : 7] -Input [2]: [d_date_sk#26, d_date#27] -Condition : isnotnull(d_date_sk#26) - -(36) Scan parquet default.date_dim -Output [2]: [d_date#28, d_week_seq#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#28, d_week_seq#29] - -(38) Filter [codegen id : 6] -Input [2]: [d_date#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND (d_week_seq#29 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) - -(39) Project [codegen id : 6] -Output [1]: [d_date#28] -Input [2]: [d_date#28, d_week_seq#29] - -(40) BroadcastExchange -Input [1]: [d_date#28] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#30] - -(41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#27] -Right keys [1]: [d_date#28] -Join condition: None - -(42) Project [codegen id : 7] +(33) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_date#27] -(43) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] - -(44) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#26] Join condition: None -(45) Project [codegen id : 8] +(35) Project [codegen id : 8] Output [2]: [cs_ext_sales_price#22, i_item_id#25] Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] -(46) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 8] Input [2]: [cs_ext_sales_price#22, i_item_id#25] Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#25, sum#33] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#25, sum#28] -(47) Exchange -Input [2]: [i_item_id#25, sum#33] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#34] +(37) Exchange +Input [2]: [i_item_id#25, sum#28] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#29] -(48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#25, sum#33] +(38) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#25, sum#28] Keys [1]: [i_item_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] -Results [2]: [i_item_id#25 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#25 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] -(49) Filter [codegen id : 9] -Input [2]: [item_id#36, cs_item_rev#37] -Condition : isnotnull(cs_item_rev#37) +(39) Filter [codegen id : 9] +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) -(50) BroadcastExchange -Input [2]: [item_id#36, cs_item_rev#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] +(40) BroadcastExchange +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] -(51) BroadcastHashJoin [codegen id : 15] +(41) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#36] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) -(52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] -Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] +(42) Project [codegen id : 15] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] -(53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +(43) Scan parquet default.web_sales +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +(44) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_item_sk#39) +(45) Filter [codegen id : 13] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) -(56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#42, i_item_id#43] +(46) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#37, i_item_id#38] -(57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#42] +(47) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#37] Join condition: None -(58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43] -Input [5]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_sk#42, i_item_id#43] +(48) Project [codegen id : 13] +Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38] +Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#37, i_item_id#38] -(59) Scan parquet default.date_dim -Output [2]: [d_date_sk#44, d_date#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#44, d_date#45] - -(61) Filter [codegen id : 12] -Input [2]: [d_date_sk#44, d_date#45] -Condition : isnotnull(d_date_sk#44) - -(62) Scan parquet default.date_dim -Output [2]: [d_date#46, d_week_seq#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct +(49) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#39] -(63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#46, d_week_seq#47] - -(64) Filter [codegen id : 11] -Input [2]: [d_date#46, d_week_seq#47] -Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) - -(65) Project [codegen id : 11] -Output [1]: [d_date#46] -Input [2]: [d_date#46, d_week_seq#47] - -(66) BroadcastExchange -Input [1]: [d_date#46] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#48] - -(67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#45] -Right keys [1]: [d_date#46] -Join condition: None - -(68) Project [codegen id : 12] -Output [1]: [d_date_sk#44] -Input [2]: [d_date_sk#44, d_date#45] - -(69) BroadcastExchange -Input [1]: [d_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] - -(70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#41] -Right keys [1]: [d_date_sk#44] +(50) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#39] Join condition: None -(71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#40, i_item_id#43] -Input [4]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43, d_date_sk#44] - -(72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#40, i_item_id#43] -Keys [1]: [i_item_id#43] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum#50] -Results [2]: [i_item_id#43, sum#51] - -(73) Exchange -Input [2]: [i_item_id#43, sum#51] -Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#52] - -(74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#43, sum#51] -Keys [1]: [i_item_id#43] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] -Results [2]: [i_item_id#43 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] - -(75) Filter [codegen id : 14] -Input [2]: [item_id#54, ws_item_rev#55] -Condition : isnotnull(ws_item_rev#55) - -(76) BroadcastExchange -Input [2]: [item_id#54, ws_item_rev#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] - -(77) BroadcastHashJoin [codegen id : 15] +(51) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#35, i_item_id#38] +Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38, d_date_sk#39] + +(52) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#35, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#38, sum#41] + +(53) Exchange +Input [2]: [i_item_id#38, sum#41] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [id=#42] + +(54) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#38, sum#41] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#38 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] + +(55) Filter [codegen id : 14] +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) + +(56) BroadcastExchange +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] + +(57) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#54] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) -(78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] +(58) Project [codegen id : 15] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] -(79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +(59) TakeOrderedAndProject +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (83) -+- * Filter (82) - +- * ColumnarToRow (81) - +- Scan parquet default.date_dim (80) +* Project (63) ++- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet default.date_dim (60) -(80) Scan parquet default.date_dim -Output [2]: [d_date#61, d_week_seq#62] +(60) Scan parquet default.date_dim +Output [2]: [d_date#51, d_week_seq#52] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] - -(82) Filter [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] -Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) - -(83) Project [codegen id : 1] -Output [1]: [d_week_seq#62] -Input [2]: [d_date#61, d_week_seq#62] +(61) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +(62) Filter [codegen id : 1] +Input [2]: [d_date#51, d_week_seq#52] +Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03)) -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +(63) Project [codegen id : 1] +Output [1]: [d_week_seq#52] +Input [2]: [d_date#51, d_week_seq#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index b2e184c7e279..1fb6832ea9b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -69,30 +69,14 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Project [d_date] - Filter [d_week_seq] - ReusedSubquery [d_week_seq] #1 - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -106,20 +90,4 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (12) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (11) - Project [d_date] - Filter [d_week_seq] - ReusedSubquery [d_week_seq] #1 - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ReusedExchange [d_date_sk] #3 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 ef5f915e1b17..499046b0d1f3 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,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (213) -+- * HashAggregate (212) - +- Exchange (211) - +- * HashAggregate (210) - +- Union (209) +TakeOrderedAndProject (157) ++- * HashAggregate (156) + +- Exchange (155) + +- * HashAggregate (154) + +- Union (153) :- * HashAggregate (132) : +- Exchange (131) : +- * HashAggregate (130) @@ -136,82 +136,26 @@ TakeOrderedAndProject (213) : : : +- 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) + :- * HashAggregate (137) + : +- Exchange (136) + : +- * HashAggregate (135) + : +- * HashAggregate (134) + : +- ReusedExchange (133) + :- * HashAggregate (142) + : +- Exchange (141) + : +- * HashAggregate (140) + : +- * HashAggregate (139) + : +- ReusedExchange (138) + :- * HashAggregate (147) + : +- Exchange (146) + : +- * HashAggregate (145) + : +- * HashAggregate (144) + : +- ReusedExchange (143) + +- * HashAggregate (152) + +- Exchange (151) + +- * HashAggregate (150) + +- * HashAggregate (149) + +- ReusedExchange (148) (1) Scan parquet default.store_sales @@ -816,581 +760,325 @@ Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] - -(134) HashAggregate [codegen id : 186] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] -Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] -Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] - -(135) Filter [codegen id : 186] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(136) Project [codegen id : 186] -Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] - -(137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] - -(138) HashAggregate [codegen id : 232] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] -Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] -Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] - -(139) Filter [codegen id : 232] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(140) Project [codegen id : 232] -Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] - -(141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] - -(142) HashAggregate [codegen id : 278] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] -Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] -Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] - -(143) Filter [codegen id : 278] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(144) Project [codegen id : 278] -Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] - -(145) Union - -(146) HashAggregate [codegen id : 279] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166] -Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] - -(147) Exchange -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] +Output [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, sum#136] -(148) HashAggregate [codegen id : 280] -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +(134) HashAggregate [codegen id : 280] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, sum#136] Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] -Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] +Aggregate Attributes [2]: [sum(sales#68)#137, sum(number_sales#69)#138] +Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#137 AS sum_sales#129, sum(number_sales#69)#138 AS number_sales#130] -(149) HashAggregate [codegen id : 280] +(135) HashAggregate [codegen id : 280] Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Aggregate Attributes [3]: [sum#139, isEmpty#140, sum#141] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144] -(150) Exchange -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] +(136) Exchange +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#145] -(151) HashAggregate [codegen id : 281] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +(137) HashAggregate [codegen id : 281] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144] Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184] - -(152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] - -(153) HashAggregate [codegen id : 327] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] -Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] -Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] - -(154) Filter [codegen id : 327] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(155) Project [codegen id : 327] -Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] - -(156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] - -(157) HashAggregate [codegen id : 373] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] -Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] -Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] - -(158) Filter [codegen id : 373] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(159) Project [codegen id : 373] -Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] - -(160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] - -(161) HashAggregate [codegen id : 419] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] -Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] -Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] - -(162) Filter [codegen id : 419] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(163) Project [codegen id : 419] -Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] - -(164) Union - -(165) HashAggregate [codegen id : 420] -Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220] -Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Aggregate Attributes [2]: [sum(sum_sales#129)#146, sum(number_sales#130)#147] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#148, sum(sum_sales#129)#146 AS sum(sum_sales)#149, sum(number_sales#130)#147 AS sum(number_sales)#150] -(166) Exchange -Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] -Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] +(138) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] -(167) HashAggregate [codegen id : 421] -Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] -Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +(139) HashAggregate [codegen id : 421] +Input [7]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] +Keys [4]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153] Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] -Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] +Aggregate Attributes [2]: [sum(sales#68)#157, sum(number_sales#69)#158] +Results [4]: [channel#73, i_brand_id#151, sum(sales#68)#157 AS sum_sales#129, sum(number_sales#69)#158 AS number_sales#130] -(168) HashAggregate [codegen id : 421] -Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] -Keys [2]: [channel#73, i_brand_id#185] +(140) HashAggregate [codegen id : 421] +Input [4]: [channel#73, i_brand_id#151, sum_sales#129, number_sales#130] +Keys [2]: [channel#73, i_brand_id#151] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] -Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Aggregate Attributes [3]: [sum#159, isEmpty#160, sum#161] +Results [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164] -(169) Exchange -Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] -Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] +(141) Exchange +Input [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164] +Arguments: hashpartitioning(channel#73, i_brand_id#151, 5), ENSURE_REQUIREMENTS, [id=#165] -(170) HashAggregate [codegen id : 422] -Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] -Keys [2]: [channel#73, i_brand_id#185] +(142) HashAggregate [codegen id : 422] +Input [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164] +Keys [2]: [channel#73, i_brand_id#151] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235] -Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239] - -(171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] - -(172) HashAggregate [codegen id : 468] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] -Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] -Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] - -(173) Filter [codegen id : 468] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(174) Project [codegen id : 468] -Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] - -(175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] - -(176) HashAggregate [codegen id : 514] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] -Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] -Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] - -(177) Filter [codegen id : 514] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(178) Project [codegen id : 514] -Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] - -(179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] - -(180) HashAggregate [codegen id : 560] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] -Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] -Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] - -(181) Filter [codegen id : 560] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(182) Project [codegen id : 560] -Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] - -(183) Union - -(184) HashAggregate [codegen id : 561] -Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275] -Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Aggregate Attributes [2]: [sum(sum_sales#129)#166, sum(number_sales#130)#167] +Results [6]: [channel#73, i_brand_id#151, null AS i_class_id#168, null AS i_category_id#169, sum(sum_sales#129)#166 AS sum(sum_sales)#170, sum(number_sales#130)#167 AS sum(number_sales)#171] -(185) Exchange -Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] -Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] +(143) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174, sum#175, isEmpty#176, sum#177] -(186) HashAggregate [codegen id : 562] -Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] -Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +(144) HashAggregate [codegen id : 562] +Input [7]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174, sum#175, isEmpty#176, sum#177] +Keys [4]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174] Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] -Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] +Aggregate Attributes [2]: [sum(sales#68)#178, sum(number_sales#69)#179] +Results [3]: [channel#73, sum(sales#68)#178 AS sum_sales#129, sum(number_sales#69)#179 AS number_sales#130] -(187) HashAggregate [codegen id : 562] +(145) HashAggregate [codegen id : 562] Input [3]: [channel#73, sum_sales#129, number_sales#130] Keys [1]: [channel#73] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284] -Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Aggregate Attributes [3]: [sum#180, isEmpty#181, sum#182] +Results [4]: [channel#73, sum#183, isEmpty#184, sum#185] -(188) Exchange -Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] -Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] +(146) Exchange +Input [4]: [channel#73, sum#183, isEmpty#184, sum#185] +Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#186] -(189) HashAggregate [codegen id : 563] -Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +(147) HashAggregate [codegen id : 563] +Input [4]: [channel#73, sum#183, isEmpty#184, sum#185] Keys [1]: [channel#73] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290] -Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295] - -(190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] - -(191) HashAggregate [codegen id : 609] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] -Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] -Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] - -(192) Filter [codegen id : 609] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(193) Project [codegen id : 609] -Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] - -(194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] - -(195) HashAggregate [codegen id : 655] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] -Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] -Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] - -(196) Filter [codegen id : 655] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(197) Project [codegen id : 655] -Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] - -(198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] - -(199) HashAggregate [codegen id : 701] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] -Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] -Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] - -(200) Filter [codegen id : 701] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(201) Project [codegen id : 701] -Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] - -(202) Union - -(203) HashAggregate [codegen id : 702] -Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331] -Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Aggregate Attributes [2]: [sum(sum_sales#129)#187, sum(number_sales#130)#188] +Results [6]: [channel#73, null AS i_brand_id#189, null AS i_class_id#190, null AS i_category_id#191, sum(sum_sales#129)#187 AS sum(sum_sales)#192, sum(number_sales#130)#188 AS sum(number_sales)#193] -(204) Exchange -Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] -Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] +(148) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196, sum#197, isEmpty#198, sum#199] -(205) HashAggregate [codegen id : 703] -Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] -Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +(149) HashAggregate [codegen id : 703] +Input [7]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196, sum#197, isEmpty#198, sum#199] +Keys [4]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196] Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] -Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] +Aggregate Attributes [2]: [sum(sales#68)#200, sum(number_sales#69)#201] +Results [2]: [sum(sales#68)#200 AS sum_sales#129, sum(number_sales#69)#201 AS number_sales#130] -(206) HashAggregate [codegen id : 703] +(150) HashAggregate [codegen id : 703] Input [2]: [sum_sales#129, number_sales#130] Keys: [] Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340] -Results [3]: [sum#341, isEmpty#342, sum#343] +Aggregate Attributes [3]: [sum#202, isEmpty#203, sum#204] +Results [3]: [sum#205, isEmpty#206, sum#207] -(207) Exchange -Input [3]: [sum#341, isEmpty#342, sum#343] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] +(151) Exchange +Input [3]: [sum#205, isEmpty#206, sum#207] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] -(208) HashAggregate [codegen id : 704] -Input [3]: [sum#341, isEmpty#342, sum#343] +(152) HashAggregate [codegen id : 704] +Input [3]: [sum#205, isEmpty#206, sum#207] Keys: [] Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346] -Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352] +Aggregate Attributes [2]: [sum(sum_sales#129)#209, sum(number_sales#130)#210] +Results [6]: [null AS channel#211, null AS i_brand_id#212, null AS i_class_id#213, null AS i_category_id#214, sum(sum_sales#129)#209 AS sum(sum_sales)#215, sum(number_sales#130)#210 AS sum(number_sales)#216] -(209) Union +(153) Union -(210) HashAggregate [codegen id : 705] +(154) HashAggregate [codegen id : 705] Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -(211) Exchange +(155) Exchange Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#217] -(212) HashAggregate [codegen id : 706] +(156) HashAggregate [codegen id : 706] Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -(213) TakeOrderedAndProject +(157) TakeOrderedAndProject Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] -* HashAggregate (240) -+- Exchange (239) - +- * HashAggregate (238) - +- Union (237) - :- * Project (222) - : +- * BroadcastHashJoin Inner BuildRight (221) - : :- * ColumnarToRow (215) - : : +- Scan parquet default.store_sales (214) - : +- BroadcastExchange (220) - : +- * Project (219) - : +- * Filter (218) - : +- * ColumnarToRow (217) - : +- Scan parquet default.date_dim (216) - :- * Project (231) - : +- * BroadcastHashJoin Inner BuildRight (230) - : :- * ColumnarToRow (224) - : : +- Scan parquet default.catalog_sales (223) - : +- BroadcastExchange (229) - : +- * Project (228) - : +- * Filter (227) - : +- * ColumnarToRow (226) - : +- Scan parquet default.date_dim (225) - +- * Project (236) - +- * BroadcastHashJoin Inner BuildRight (235) - :- * ColumnarToRow (233) - : +- Scan parquet default.web_sales (232) - +- ReusedExchange (234) - - -(214) Scan parquet default.store_sales -Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] +* HashAggregate (184) ++- Exchange (183) + +- * HashAggregate (182) + +- Union (181) + :- * Project (166) + : +- * BroadcastHashJoin Inner BuildRight (165) + : :- * ColumnarToRow (159) + : : +- Scan parquet default.store_sales (158) + : +- BroadcastExchange (164) + : +- * Project (163) + : +- * Filter (162) + : +- * ColumnarToRow (161) + : +- Scan parquet default.date_dim (160) + :- * Project (175) + : +- * BroadcastHashJoin Inner BuildRight (174) + : :- * ColumnarToRow (168) + : : +- Scan parquet default.catalog_sales (167) + : +- BroadcastExchange (173) + : +- * Project (172) + : +- * Filter (171) + : +- * ColumnarToRow (170) + : +- Scan parquet default.date_dim (169) + +- * Project (180) + +- * BroadcastHashJoin Inner BuildRight (179) + :- * ColumnarToRow (177) + : +- Scan parquet default.web_sales (176) + +- ReusedExchange (178) + + +(158) Scan parquet default.store_sales +Output [3]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)] +PartitionFilters: [isnotnull(ss_sold_date_sk#220), dynamicpruningexpression(ss_sold_date_sk#220 IN dynamicpruning#221)] ReadSchema: struct -(215) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] +(159) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220] -(216) Scan parquet default.date_dim -Output [2]: [d_date_sk#358, d_year#359] +(160) Scan parquet default.date_dim +Output [2]: [d_date_sk#222, d_year#223] 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 -(217) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#358, d_year#359] +(161) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#222, d_year#223] -(218) Filter [codegen id : 1] -Input [2]: [d_date_sk#358, d_year#359] -Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358)) +(162) Filter [codegen id : 1] +Input [2]: [d_date_sk#222, d_year#223] +Condition : (((isnotnull(d_year#223) AND (d_year#223 >= 1999)) AND (d_year#223 <= 2001)) AND isnotnull(d_date_sk#222)) -(219) Project [codegen id : 1] -Output [1]: [d_date_sk#358] -Input [2]: [d_date_sk#358, d_year#359] +(163) Project [codegen id : 1] +Output [1]: [d_date_sk#222] +Input [2]: [d_date_sk#222, d_year#223] -(220) BroadcastExchange -Input [1]: [d_date_sk#358] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360] +(164) BroadcastExchange +Input [1]: [d_date_sk#222] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#224] -(221) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#356] -Right keys [1]: [d_date_sk#358] +(165) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#220] +Right keys [1]: [d_date_sk#222] Join condition: None -(222) Project [codegen id : 2] -Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362] -Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358] +(166) Project [codegen id : 2] +Output [2]: [ss_quantity#218 AS quantity#225, ss_list_price#219 AS list_price#226] +Input [4]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220, d_date_sk#222] -(223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] +(167) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)] +PartitionFilters: [isnotnull(cs_sold_date_sk#229), dynamicpruningexpression(cs_sold_date_sk#229 IN dynamicpruning#230)] ReadSchema: struct -(224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] +(168) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229] -(225) Scan parquet default.date_dim -Output [2]: [d_date_sk#367, d_year#368] +(169) Scan parquet default.date_dim +Output [2]: [d_date_sk#231, d_year#232] 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 -(226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#367, d_year#368] +(170) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#231, d_year#232] -(227) Filter [codegen id : 3] -Input [2]: [d_date_sk#367, d_year#368] -Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367)) +(171) Filter [codegen id : 3] +Input [2]: [d_date_sk#231, d_year#232] +Condition : (((isnotnull(d_year#232) AND (d_year#232 >= 1998)) AND (d_year#232 <= 2000)) AND isnotnull(d_date_sk#231)) -(228) Project [codegen id : 3] -Output [1]: [d_date_sk#367] -Input [2]: [d_date_sk#367, d_year#368] +(172) Project [codegen id : 3] +Output [1]: [d_date_sk#231] +Input [2]: [d_date_sk#231, d_year#232] -(229) BroadcastExchange -Input [1]: [d_date_sk#367] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369] +(173) BroadcastExchange +Input [1]: [d_date_sk#231] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#233] -(230) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#365] -Right keys [1]: [d_date_sk#367] +(174) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#229] +Right keys [1]: [d_date_sk#231] Join condition: None -(231) Project [codegen id : 4] -Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371] -Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367] +(175) Project [codegen id : 4] +Output [2]: [cs_quantity#227 AS quantity#234, cs_list_price#228 AS list_price#235] +Input [4]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229, d_date_sk#231] -(232) Scan parquet default.web_sales -Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] +(176) Scan parquet default.web_sales +Output [3]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)] +PartitionFilters: [isnotnull(ws_sold_date_sk#238), dynamicpruningexpression(ws_sold_date_sk#238 IN dynamicpruning#230)] ReadSchema: struct -(233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] +(177) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238] -(234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#375] +(178) ReusedExchange [Reuses operator id: 173] +Output [1]: [d_date_sk#239] -(235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#374] -Right keys [1]: [d_date_sk#375] +(179) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#238] +Right keys [1]: [d_date_sk#239] Join condition: None -(236) Project [codegen id : 6] -Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377] -Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375] +(180) Project [codegen id : 6] +Output [2]: [ws_quantity#236 AS quantity#240, ws_list_price#237 AS list_price#241] +Input [4]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238, d_date_sk#239] -(237) Union +(181) Union -(238) HashAggregate [codegen id : 7] -Input [2]: [quantity#361, list_price#362] +(182) HashAggregate [codegen id : 7] +Input [2]: [quantity#225, list_price#226] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#378, count#379] -Results [2]: [sum#380, count#381] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#242, count#243] +Results [2]: [sum#244, count#245] -(239) Exchange -Input [2]: [sum#380, count#381] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382] +(183) Exchange +Input [2]: [sum#244, count#245] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#246] -(240) HashAggregate [codegen id : 8] -Input [2]: [sum#380, count#381] +(184) HashAggregate [codegen id : 8] +Input [2]: [sum#244, count#245] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))#247] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))#247 AS average_sales#248] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357 -ReusedExchange (241) +Subquery:2 Hosting operator id = 158 Hosting Expression = ss_sold_date_sk#220 IN dynamicpruning#221 +ReusedExchange (185) -(241) ReusedExchange [Reuses operator id: 220] -Output [1]: [d_date_sk#358] +(185) ReusedExchange [Reuses operator id: 164] +Output [1]: [d_date_sk#222] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366 -ReusedExchange (242) +Subquery:3 Hosting operator id = 167 Hosting Expression = cs_sold_date_sk#229 IN dynamicpruning#230 +ReusedExchange (186) -(242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#367] +(186) ReusedExchange [Reuses operator id: 173] +Output [1]: [d_date_sk#231] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366 +Subquery:4 Hosting operator id = 176 Hosting Expression = ws_sold_date_sk#238 IN dynamicpruning#230 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (243) +ReusedExchange (187) -(243) ReusedExchange [Reuses operator id: 72] +(187) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#49] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -ReusedExchange (244) +ReusedExchange (188) -(244) ReusedExchange [Reuses operator id: 16] +(188) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#14] Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 @@ -1405,28 +1093,4 @@ Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subque Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] - 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 0a9fa7cbf349..6a4204810175 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 @@ -300,131 +300,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num 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] #28 - WholeStageCodegen (279) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (186) - 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] #3 - 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] #29 - WholeStageCodegen (232) - 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] #3 - 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] #30 - WholeStageCodegen (278) - 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] #3 - 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] #31 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28 WholeStageCodegen (422) 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] #32 + Exchange [channel,i_brand_id] #29 WholeStageCodegen (421) 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] #33 - WholeStageCodegen (420) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (327) - 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] #3 - 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] #29 - WholeStageCodegen (373) - 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] #3 - 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] #30 - WholeStageCodegen (419) - 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] #3 - 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] #31 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28 WholeStageCodegen (563) 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] #34 + Exchange [channel] #30 WholeStageCodegen (562) 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] #35 - WholeStageCodegen (561) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (468) - 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] #3 - 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] #29 - WholeStageCodegen (514) - 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] #3 - 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] #30 - WholeStageCodegen (560) - 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] #3 - 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] #31 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28 WholeStageCodegen (704) 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 #36 + Exchange #31 WholeStageCodegen (703) 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] #37 - WholeStageCodegen (702) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (609) - 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] #3 - 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] #29 - WholeStageCodegen (655) - 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] #3 - 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] #30 - WholeStageCodegen (701) - 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] #3 - 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] #31 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 755973f79e49..844ae4b303d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (194) -+- * HashAggregate (193) - +- Exchange (192) - +- * HashAggregate (191) - +- Union (190) +TakeOrderedAndProject (138) ++- * HashAggregate (137) + +- Exchange (136) + +- * HashAggregate (135) + +- Union (134) :- * HashAggregate (113) : +- Exchange (112) : +- * HashAggregate (111) @@ -117,82 +117,26 @@ TakeOrderedAndProject (194) : : : +- ReusedExchange (97) : : +- ReusedExchange (99) : +- ReusedExchange (102) - :- * HashAggregate (132) - : +- Exchange (131) - : +- * HashAggregate (130) - : +- * HashAggregate (129) - : +- Exchange (128) - : +- * HashAggregate (127) - : +- Union (126) - : :- * Project (117) - : : +- * Filter (116) - : : +- * HashAggregate (115) - : : +- ReusedExchange (114) - : :- * Project (121) - : : +- * Filter (120) - : : +- * HashAggregate (119) - : : +- ReusedExchange (118) - : +- * Project (125) - : +- * Filter (124) - : +- * HashAggregate (123) - : +- ReusedExchange (122) - :- * 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 (118) + : +- Exchange (117) + : +- * HashAggregate (116) + : +- * HashAggregate (115) + : +- ReusedExchange (114) + :- * HashAggregate (123) + : +- Exchange (122) + : +- * HashAggregate (121) + : +- * HashAggregate (120) + : +- ReusedExchange (119) + :- * HashAggregate (128) + : +- Exchange (127) + : +- * HashAggregate (126) + : +- * HashAggregate (125) + : +- ReusedExchange (124) + +- * HashAggregate (133) + +- Exchange (132) + +- * HashAggregate (131) + +- * HashAggregate (130) + +- ReusedExchange (129) (1) Scan parquet default.store_sales @@ -721,581 +665,325 @@ Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] - -(115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] -Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] - -(116) Filter [codegen id : 106] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(117) Project [codegen id : 106] -Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] - -(118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] - -(119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] -Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] -Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] - -(120) Filter [codegen id : 132] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(121) Project [codegen id : 132] -Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] - -(122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] - -(123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] -Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155] -Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] - -(124) Filter [codegen id : 158] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(125) Project [codegen id : 158] -Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] - -(126) Union - -(127) HashAggregate [codegen id : 159] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] -Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] - -(128) Exchange -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163] +Output [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, sum#129] -(129) HashAggregate [codegen id : 160] -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +(115) HashAggregate [codegen id : 160] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, sum#129] Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165] -Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123] +Aggregate Attributes [2]: [sum(sales#63)#130, sum(number_sales#64)#131] +Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#130 AS sum_sales#122, sum(number_sales#64)#131 AS number_sales#123] -(130) HashAggregate [codegen id : 160] +(116) HashAggregate [codegen id : 160] Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Aggregate Attributes [3]: [sum#132, isEmpty#133, sum#134] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137] -(131) Exchange -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172] +(117) Exchange +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#138] -(132) HashAggregate [codegen id : 161] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +(118) HashAggregate [codegen id : 161] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137] Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177] - -(133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] - -(134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] -Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] -Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] - -(135) Filter [codegen id : 187] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(136) Project [codegen id : 187] -Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] - -(137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] - -(138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] -Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] -Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] - -(139) Filter [codegen id : 213] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(140) Project [codegen id : 213] -Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] - -(141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] - -(142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] -Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209] -Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] - -(143) Filter [codegen id : 239] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(144) Project [codegen id : 239] -Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] - -(145) Union - -(146) HashAggregate [codegen id : 240] -Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213] -Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Aggregate Attributes [2]: [sum(sum_sales#122)#139, sum(number_sales#123)#140] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#141, sum(sum_sales#122)#139 AS sum(sum_sales)#142, sum(number_sales#123)#140 AS sum(number_sales)#143] -(147) Exchange -Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] -Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217] +(119) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146, sum#147, isEmpty#148, sum#149] -(148) HashAggregate [codegen id : 241] -Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] -Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +(120) HashAggregate [codegen id : 241] +Input [7]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146, sum#147, isEmpty#148, sum#149] +Keys [4]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146] Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219] -Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123] +Aggregate Attributes [2]: [sum(sales#63)#150, sum(number_sales#64)#151] +Results [4]: [channel#68, i_brand_id#144, sum(sales#63)#150 AS sum_sales#122, sum(number_sales#64)#151 AS number_sales#123] -(149) HashAggregate [codegen id : 241] -Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123] -Keys [2]: [channel#68, i_brand_id#178] +(121) HashAggregate [codegen id : 241] +Input [4]: [channel#68, i_brand_id#144, sum_sales#122, number_sales#123] +Keys [2]: [channel#68, i_brand_id#144] Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222] -Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] +Results [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157] -(150) Exchange -Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] -Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226] +(122) Exchange +Input [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157] +Arguments: hashpartitioning(channel#68, i_brand_id#144, 5), ENSURE_REQUIREMENTS, [id=#158] -(151) HashAggregate [codegen id : 242] -Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] -Keys [2]: [channel#68, i_brand_id#178] +(123) HashAggregate [codegen id : 242] +Input [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157] +Keys [2]: [channel#68, i_brand_id#144] Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228] -Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232] - -(152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] - -(153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] -Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] -Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] - -(154) Filter [codegen id : 268] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(155) Project [codegen id : 268] -Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] - -(156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] - -(157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] -Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] -Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] - -(158) Filter [codegen id : 294] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(159) Project [codegen id : 294] -Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] - -(160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] - -(161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] -Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264] -Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] - -(162) Filter [codegen id : 320] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(163) Project [codegen id : 320] -Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] - -(164) Union - -(165) HashAggregate [codegen id : 321] -Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268] -Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Aggregate Attributes [2]: [sum(sum_sales#122)#159, sum(number_sales#123)#160] +Results [6]: [channel#68, i_brand_id#144, null AS i_class_id#161, null AS i_category_id#162, sum(sum_sales#122)#159 AS sum(sum_sales)#163, sum(number_sales#123)#160 AS sum(number_sales)#164] -(166) Exchange -Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] -Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272] +(124) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167, sum#168, isEmpty#169, sum#170] -(167) HashAggregate [codegen id : 322] -Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] -Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +(125) HashAggregate [codegen id : 322] +Input [7]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167, sum#168, isEmpty#169, sum#170] +Keys [4]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167] Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274] -Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123] +Aggregate Attributes [2]: [sum(sales#63)#171, sum(number_sales#64)#172] +Results [3]: [channel#68, sum(sales#63)#171 AS sum_sales#122, sum(number_sales#64)#172 AS number_sales#123] -(168) HashAggregate [codegen id : 322] +(126) HashAggregate [codegen id : 322] Input [3]: [channel#68, sum_sales#122, number_sales#123] Keys [1]: [channel#68] Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277] -Results [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] +Results [4]: [channel#68, sum#176, isEmpty#177, sum#178] -(169) Exchange -Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] -Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281] +(127) Exchange +Input [4]: [channel#68, sum#176, isEmpty#177, sum#178] +Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#179] -(170) HashAggregate [codegen id : 323] -Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +(128) HashAggregate [codegen id : 323] +Input [4]: [channel#68, sum#176, isEmpty#177, sum#178] Keys [1]: [channel#68] Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283] -Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288] - -(171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] - -(172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] -Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] -Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] - -(173) Filter [codegen id : 349] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(174) Project [codegen id : 349] -Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] - -(175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] - -(176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] -Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] -Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] - -(177) Filter [codegen id : 375] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(178) Project [codegen id : 375] -Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] - -(179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] - -(180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] -Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320] -Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] - -(181) Filter [codegen id : 401] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(182) Project [codegen id : 401] -Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] - -(183) Union - -(184) HashAggregate [codegen id : 402] -Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324] -Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Aggregate Attributes [2]: [sum(sum_sales#122)#180, sum(number_sales#123)#181] +Results [6]: [channel#68, null AS i_brand_id#182, null AS i_class_id#183, null AS i_category_id#184, sum(sum_sales#122)#180 AS sum(sum_sales)#185, sum(number_sales#123)#181 AS sum(number_sales)#186] -(185) Exchange -Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] -Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328] +(129) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189, sum#190, isEmpty#191, sum#192] -(186) HashAggregate [codegen id : 403] -Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] -Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +(130) HashAggregate [codegen id : 403] +Input [7]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189, sum#190, isEmpty#191, sum#192] +Keys [4]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189] Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330] -Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123] +Aggregate Attributes [2]: [sum(sales#63)#193, sum(number_sales#64)#194] +Results [2]: [sum(sales#63)#193 AS sum_sales#122, sum(number_sales#64)#194 AS number_sales#123] -(187) HashAggregate [codegen id : 403] +(131) HashAggregate [codegen id : 403] Input [2]: [sum_sales#122, number_sales#123] Keys: [] Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333] -Results [3]: [sum#334, isEmpty#335, sum#336] +Aggregate Attributes [3]: [sum#195, isEmpty#196, sum#197] +Results [3]: [sum#198, isEmpty#199, sum#200] -(188) Exchange -Input [3]: [sum#334, isEmpty#335, sum#336] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337] +(132) Exchange +Input [3]: [sum#198, isEmpty#199, sum#200] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#201] -(189) HashAggregate [codegen id : 404] -Input [3]: [sum#334, isEmpty#335, sum#336] +(133) HashAggregate [codegen id : 404] +Input [3]: [sum#198, isEmpty#199, sum#200] Keys: [] Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339] -Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345] +Aggregate Attributes [2]: [sum(sum_sales#122)#202, sum(number_sales#123)#203] +Results [6]: [null AS channel#204, null AS i_brand_id#205, null AS i_class_id#206, null AS i_category_id#207, sum(sum_sales#122)#202 AS sum(sum_sales)#208, sum(number_sales#123)#203 AS sum(number_sales)#209] -(190) Union +(134) Union -(191) HashAggregate [codegen id : 405] +(135) HashAggregate [codegen id : 405] Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -(192) Exchange +(136) Exchange Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#210] -(193) HashAggregate [codegen id : 406] +(137) HashAggregate [codegen id : 406] Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -(194) TakeOrderedAndProject +(138) TakeOrderedAndProject Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] ===== Subqueries ===== Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* HashAggregate (221) -+- Exchange (220) - +- * HashAggregate (219) - +- Union (218) - :- * Project (203) - : +- * BroadcastHashJoin Inner BuildRight (202) - : :- * ColumnarToRow (196) - : : +- Scan parquet default.store_sales (195) - : +- BroadcastExchange (201) - : +- * Project (200) - : +- * Filter (199) - : +- * ColumnarToRow (198) - : +- Scan parquet default.date_dim (197) - :- * Project (212) - : +- * BroadcastHashJoin Inner BuildRight (211) - : :- * ColumnarToRow (205) - : : +- Scan parquet default.catalog_sales (204) - : +- BroadcastExchange (210) - : +- * Project (209) - : +- * Filter (208) - : +- * ColumnarToRow (207) - : +- Scan parquet default.date_dim (206) - +- * Project (217) - +- * BroadcastHashJoin Inner BuildRight (216) - :- * ColumnarToRow (214) - : +- Scan parquet default.web_sales (213) - +- ReusedExchange (215) - - -(195) Scan parquet default.store_sales -Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] +* HashAggregate (165) ++- Exchange (164) + +- * HashAggregate (163) + +- Union (162) + :- * Project (147) + : +- * BroadcastHashJoin Inner BuildRight (146) + : :- * ColumnarToRow (140) + : : +- Scan parquet default.store_sales (139) + : +- BroadcastExchange (145) + : +- * Project (144) + : +- * Filter (143) + : +- * ColumnarToRow (142) + : +- Scan parquet default.date_dim (141) + :- * Project (156) + : +- * BroadcastHashJoin Inner BuildRight (155) + : :- * ColumnarToRow (149) + : : +- Scan parquet default.catalog_sales (148) + : +- BroadcastExchange (154) + : +- * Project (153) + : +- * Filter (152) + : +- * ColumnarToRow (151) + : +- Scan parquet default.date_dim (150) + +- * Project (161) + +- * BroadcastHashJoin Inner BuildRight (160) + :- * ColumnarToRow (158) + : +- Scan parquet default.web_sales (157) + +- ReusedExchange (159) + + +(139) Scan parquet default.store_sales +Output [3]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)] +PartitionFilters: [isnotnull(ss_sold_date_sk#213), dynamicpruningexpression(ss_sold_date_sk#213 IN dynamicpruning#214)] ReadSchema: struct -(196) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] +(140) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213] -(197) Scan parquet default.date_dim -Output [2]: [d_date_sk#351, d_year#352] +(141) Scan parquet default.date_dim +Output [2]: [d_date_sk#215, d_year#216] 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 -(198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#351, d_year#352] +(142) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#215, d_year#216] -(199) Filter [codegen id : 1] -Input [2]: [d_date_sk#351, d_year#352] -Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351)) +(143) Filter [codegen id : 1] +Input [2]: [d_date_sk#215, d_year#216] +Condition : (((isnotnull(d_year#216) AND (d_year#216 >= 1999)) AND (d_year#216 <= 2001)) AND isnotnull(d_date_sk#215)) -(200) Project [codegen id : 1] -Output [1]: [d_date_sk#351] -Input [2]: [d_date_sk#351, d_year#352] +(144) Project [codegen id : 1] +Output [1]: [d_date_sk#215] +Input [2]: [d_date_sk#215, d_year#216] -(201) BroadcastExchange -Input [1]: [d_date_sk#351] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353] +(145) BroadcastExchange +Input [1]: [d_date_sk#215] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#217] -(202) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#349] -Right keys [1]: [d_date_sk#351] +(146) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#213] +Right keys [1]: [d_date_sk#215] Join condition: None -(203) Project [codegen id : 2] -Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355] -Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351] +(147) Project [codegen id : 2] +Output [2]: [ss_quantity#211 AS quantity#218, ss_list_price#212 AS list_price#219] +Input [4]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213, d_date_sk#215] -(204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] +(148) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)] +PartitionFilters: [isnotnull(cs_sold_date_sk#222), dynamicpruningexpression(cs_sold_date_sk#222 IN dynamicpruning#223)] ReadSchema: struct -(205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] +(149) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222] -(206) Scan parquet default.date_dim -Output [2]: [d_date_sk#360, d_year#361] +(150) Scan parquet default.date_dim +Output [2]: [d_date_sk#224, d_year#225] 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 -(207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#360, d_year#361] +(151) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#224, d_year#225] -(208) Filter [codegen id : 3] -Input [2]: [d_date_sk#360, d_year#361] -Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360)) +(152) Filter [codegen id : 3] +Input [2]: [d_date_sk#224, d_year#225] +Condition : (((isnotnull(d_year#225) AND (d_year#225 >= 1998)) AND (d_year#225 <= 2000)) AND isnotnull(d_date_sk#224)) -(209) Project [codegen id : 3] -Output [1]: [d_date_sk#360] -Input [2]: [d_date_sk#360, d_year#361] +(153) Project [codegen id : 3] +Output [1]: [d_date_sk#224] +Input [2]: [d_date_sk#224, d_year#225] -(210) BroadcastExchange -Input [1]: [d_date_sk#360] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362] +(154) BroadcastExchange +Input [1]: [d_date_sk#224] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#226] -(211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#358] -Right keys [1]: [d_date_sk#360] +(155) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#222] +Right keys [1]: [d_date_sk#224] Join condition: None -(212) Project [codegen id : 4] -Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364] -Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360] +(156) Project [codegen id : 4] +Output [2]: [cs_quantity#220 AS quantity#227, cs_list_price#221 AS list_price#228] +Input [4]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222, d_date_sk#224] -(213) Scan parquet default.web_sales -Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] +(157) Scan parquet default.web_sales +Output [3]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)] +PartitionFilters: [isnotnull(ws_sold_date_sk#231), dynamicpruningexpression(ws_sold_date_sk#231 IN dynamicpruning#223)] ReadSchema: struct -(214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] +(158) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231] -(215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#368] +(159) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#232] -(216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#367] -Right keys [1]: [d_date_sk#368] +(160) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#231] +Right keys [1]: [d_date_sk#232] Join condition: None -(217) Project [codegen id : 6] -Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370] -Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368] +(161) Project [codegen id : 6] +Output [2]: [ws_quantity#229 AS quantity#233, ws_list_price#230 AS list_price#234] +Input [4]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231, d_date_sk#232] -(218) Union +(162) Union -(219) HashAggregate [codegen id : 7] -Input [2]: [quantity#354, list_price#355] +(163) HashAggregate [codegen id : 7] +Input [2]: [quantity#218, list_price#219] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#371, count#372] -Results [2]: [sum#373, count#374] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#235, count#236] +Results [2]: [sum#237, count#238] -(220) Exchange -Input [2]: [sum#373, count#374] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375] +(164) Exchange +Input [2]: [sum#237, count#238] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#239] -(221) HashAggregate [codegen id : 8] -Input [2]: [sum#373, count#374] +(165) HashAggregate [codegen id : 8] +Input [2]: [sum#237, count#238] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))#240] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))#240 AS average_sales#241] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350 -ReusedExchange (222) +Subquery:2 Hosting operator id = 139 Hosting Expression = ss_sold_date_sk#213 IN dynamicpruning#214 +ReusedExchange (166) -(222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#351] +(166) ReusedExchange [Reuses operator id: 145] +Output [1]: [d_date_sk#215] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359 -ReusedExchange (223) +Subquery:3 Hosting operator id = 148 Hosting Expression = cs_sold_date_sk#222 IN dynamicpruning#223 +ReusedExchange (167) -(223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#360] +(167) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#224] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359 +Subquery:4 Hosting operator id = 157 Hosting Expression = ws_sold_date_sk#231 IN dynamicpruning#223 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (224) +ReusedExchange (168) -(224) ReusedExchange [Reuses operator id: 70] +(168) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#50] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -ReusedExchange (225) +ReusedExchange (169) -(225) ReusedExchange [Reuses operator id: 26] +(169) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#29] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 @@ -1310,28 +998,4 @@ Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subque Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index e351f9e68702..521e6d5a776e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -243,131 +243,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num 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] #21 - WholeStageCodegen (159) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (106) - 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] #3 - 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] #22 - WholeStageCodegen (132) - 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] #3 - 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] #23 - WholeStageCodegen (158) - 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] #3 - 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] #24 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21 WholeStageCodegen (242) 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] #25 + Exchange [channel,i_brand_id] #22 WholeStageCodegen (241) 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] #26 - WholeStageCodegen (240) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (187) - 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] #3 - 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] #22 - WholeStageCodegen (213) - 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] #3 - 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] #23 - WholeStageCodegen (239) - 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] #3 - 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] #24 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21 WholeStageCodegen (323) 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] #27 + Exchange [channel] #23 WholeStageCodegen (322) 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] #28 - WholeStageCodegen (321) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (268) - 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] #3 - 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] #22 - WholeStageCodegen (294) - 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] #3 - 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] #23 - WholeStageCodegen (320) - 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] #3 - 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] #24 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21 WholeStageCodegen (404) 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 #29 + Exchange #24 WholeStageCodegen (403) 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] #30 - WholeStageCodegen (402) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (349) - 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] #3 - 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] #22 - WholeStageCodegen (375) - 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] #3 - 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] #23 - WholeStageCodegen (401) - 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] #3 - 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] #24 + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21