[SPARK-32951][SQL] Foldable propagation from Aggregate

### What changes were proposed in this pull request?
This PR adds foldable propagation from `Aggregate` as per: https://github.com/apache/spark/pull/29771#discussion_r490412031

### Why are the changes needed?
This is an improvement as `Aggregate`'s `aggregateExpressions` can contain foldables that can be propagated up.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
New UT.

Closes #29816 from peter-toth/SPARK-32951-foldable-propagation-from-aggregate.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
This commit is contained in:
Peter Toth 2020-09-21 21:43:17 -07:00 committed by Dongjoon Hyun
parent 5440ea84ee
commit f03c03576a
22 changed files with 799 additions and 776 deletions

View file

@ -633,11 +633,16 @@ object FoldablePropagation extends Rule[LogicalPlan] {
val (newChild, foldableMap) = propagateFoldables(p.child)
val newProject =
replaceFoldable(p.withNewChildren(Seq(newChild)).asInstanceOf[Project], foldableMap)
val newFoldableMap = AttributeMap(newProject.projectList.collect {
case a: Alias if a.child.foldable => (a.toAttribute, a)
})
val newFoldableMap = collectFoldables(newProject.projectList)
(newProject, newFoldableMap)
case a: Aggregate =>
val (newChild, foldableMap) = propagateFoldables(a.child)
val newAggregate =
replaceFoldable(a.withNewChildren(Seq(newChild)).asInstanceOf[Aggregate], foldableMap)
val newFoldableMap = collectFoldables(newAggregate.aggregateExpressions)
(newAggregate, newFoldableMap)
// We can not replace the attributes in `Expand.output`. If there are other non-leaf
// operators that have the `output` field, we should put them here too.
case e: Expand =>
@ -703,6 +708,12 @@ object FoldablePropagation extends Rule[LogicalPlan] {
}
}
private def collectFoldables(expressions: Seq[NamedExpression]) = {
AttributeMap(expressions.collect {
case a: Alias if a.child.foldable => (a.toAttribute, a)
})
}
/**
* List of all [[UnaryNode]]s which allow foldable propagation.
*/
@ -710,7 +721,7 @@ object FoldablePropagation extends Rule[LogicalPlan] {
// Handling `Project` is moved to `propagateFoldables`.
case _: Filter => true
case _: SubqueryAlias => true
case _: Aggregate => true
// Handling `Aggregate` is moved to `propagateFoldables`.
case _: Window => true
case _: Sample => true
case _: GlobalLimit => true

View file

@ -192,4 +192,16 @@ class FoldablePropagationSuite extends PlanTest {
val optimized = Optimize.execute(query)
comparePlans(optimized, query)
}
test("SPARK-32951: Foldable propagation from Aggregate") {
val query = testRelation
.groupBy('a)('a, sum('b).as('b), Literal(1).as('c))
.select('a, 'b, 'c)
val optimized = Optimize.execute(query.analyze)
val correctAnswer = testRelation
.groupBy('a)('a, sum('b).as('b), Literal(1).as('c))
.select('a, 'b, Literal(1).as('c)).analyze
comparePlans(optimized, correctAnswer)
}
}

View file

@ -517,15 +517,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41]
Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#43, count(1)#41 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
(86) Filter [codegen id : 39]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(87) Project [codegen id : 39]
Output [6]: [sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45]
Output [6]: [sales#42, number_sales#43, store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
(88) Scan parquet default.catalog_sales
Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49]
@ -601,15 +601,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59]
Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#61, count(1)#59 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62]
(105) Filter [codegen id : 78]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(106) Project [codegen id : 78]
Output [6]: [sales#61, number_sales#62, channel#60, i_brand_id#7, i_class_id#8, i_category_id#9]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63]
Output [6]: [sales#60, number_sales#61, catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62]
(107) Scan parquet default.web_sales
Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65]
@ -685,26 +685,26 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75]
Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#77, count(1)#75 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78]
(124) Filter [codegen id : 117]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(125) Project [codegen id : 117]
Output [6]: [sales#77, number_sales#78, channel#76, i_brand_id#7, i_class_id#8, i_category_id#9]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79]
Output [6]: [sales#76, number_sales#77, web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78]
(126) Union
(127) Expand [codegen id : 118]
Input [6]: [sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Arguments: [List(sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#43, number_sales#44, channel#42, i_brand_id#7, i_class_id#8, null, 1), List(sales#43, number_sales#44, channel#42, i_brand_id#7, null, null, 3), List(sales#43, number_sales#44, channel#42, null, null, null, 7), List(sales#43, number_sales#44, null, null, null, null, 15)], [sales#43, number_sales#44, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84]
Input [6]: [sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Arguments: [List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, null, 1), List(sales#42, number_sales#43, channel#47, i_brand_id#7, null, null, 3), List(sales#42, number_sales#43, channel#47, null, null, null, 7), List(sales#42, number_sales#43, null, null, null, null, 15)], [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84]
(128) HashAggregate [codegen id : 118]
Input [7]: [sales#43, number_sales#44, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84]
Input [7]: [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84]
Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84]
Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)]
Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)]
Aggregate Attributes [3]: [sum#85, isEmpty#86, sum#87]
Results [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90]
@ -715,9 +715,9 @@ Arguments: hashpartitioning(channel#80, i_brand_id#81, i_class_id#82, i_category
(130) HashAggregate [codegen id : 119]
Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90]
Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84]
Functions [2]: [sum(sales#43), sum(number_sales#44)]
Aggregate Attributes [2]: [sum(sales#43)#92, sum(number_sales#44)#93]
Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#43)#92 AS sum(sales)#94, sum(number_sales#44)#93 AS sum(number_sales)#95]
Functions [2]: [sum(sales#42), sum(number_sales#43)]
Aggregate Attributes [2]: [sum(sales#42)#92, sum(number_sales#43)#93]
Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#42)#92 AS sum(sales)#94, sum(number_sales#43)#93 AS sum(number_sales)#95]
(131) TakeOrderedAndProject
Input [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95]
@ -725,7 +725,7 @@ Arguments: 100, [channel#80 ASC NULLS FIRST, i_brand_id#81 ASC NULLS FIRST, i_cl
===== Subqueries =====
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#46, [id=#47]
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46]
* HashAggregate (157)
+- Exchange (156)
+- * HashAggregate (155)
@ -871,8 +871,8 @@ Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as de
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108 AS average_sales#109]
Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]

View file

@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
InputAdapter
Union
WholeStageCodegen (39)
Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #1
WholeStageCodegen (8)
@ -53,7 +53,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #17
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #2
WholeStageCodegen (38)
@ -189,10 +189,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
InputAdapter
ReusedExchange [ss_item_sk] #4
WholeStageCodegen (78)
Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #18
WholeStageCodegen (77)
@ -221,10 +221,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
InputAdapter
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
WholeStageCodegen (117)
Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #20
WholeStageCodegen (116)

View file

@ -461,15 +461,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38]
Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#40, count(1)#38 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41]
(76) Filter [codegen id : 26]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 as decimal(32,6)) > cast(Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(77) Project [codegen id : 26]
Output [6]: [sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42]
Output [6]: [sales#39, number_sales#40, store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41]
(78) Scan parquet default.catalog_sales
Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46]
@ -533,15 +533,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55]
Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#57, count(1)#55 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58]
(92) Filter [codegen id : 52]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(93) Project [codegen id : 52]
Output [6]: [sales#57, number_sales#58, channel#56, i_brand_id#6, i_class_id#7, i_category_id#8]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59]
Output [6]: [sales#56, number_sales#57, catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58]
(94) Scan parquet default.web_sales
Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61]
@ -605,26 +605,26 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70]
Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#72, count(1)#70 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73]
(108) Filter [codegen id : 78]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(109) Project [codegen id : 78]
Output [6]: [sales#72, number_sales#73, channel#71, i_brand_id#6, i_class_id#7, i_category_id#8]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74]
Output [6]: [sales#71, number_sales#72, web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73]
(110) Union
(111) Expand [codegen id : 79]
Input [6]: [sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Arguments: [List(sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#40, number_sales#41, channel#39, i_brand_id#6, i_class_id#7, null, 1), List(sales#40, number_sales#41, channel#39, i_brand_id#6, null, null, 3), List(sales#40, number_sales#41, channel#39, null, null, null, 7), List(sales#40, number_sales#41, null, null, null, null, 15)], [sales#40, number_sales#41, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79]
Input [6]: [sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Arguments: [List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, null, 1), List(sales#39, number_sales#40, channel#44, i_brand_id#6, null, null, 3), List(sales#39, number_sales#40, channel#44, null, null, null, 7), List(sales#39, number_sales#40, null, null, null, null, 15)], [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79]
(112) HashAggregate [codegen id : 79]
Input [7]: [sales#40, number_sales#41, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79]
Input [7]: [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79]
Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79]
Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)]
Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)]
Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82]
Results [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85]
@ -635,9 +635,9 @@ Arguments: hashpartitioning(channel#75, i_brand_id#76, i_class_id#77, i_category
(114) HashAggregate [codegen id : 80]
Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85]
Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79]
Functions [2]: [sum(sales#40), sum(number_sales#41)]
Aggregate Attributes [2]: [sum(sales#40)#87, sum(number_sales#41)#88]
Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#40)#87 AS sum(sales)#89, sum(number_sales#41)#88 AS sum(number_sales)#90]
Functions [2]: [sum(sales#39), sum(number_sales#40)]
Aggregate Attributes [2]: [sum(sales#39)#87, sum(number_sales#40)#88]
Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#39)#87 AS sum(sales)#89, sum(number_sales#40)#88 AS sum(number_sales)#90]
(115) TakeOrderedAndProject
Input [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90]
@ -645,7 +645,7 @@ Arguments: 100, [channel#75 ASC NULLS FIRST, i_brand_id#76 ASC NULLS FIRST, i_cl
===== Subqueries =====
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#43, [id=#44]
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43]
* HashAggregate (141)
+- Exchange (140)
+- * HashAggregate (139)
@ -791,8 +791,8 @@ Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as de
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103 AS average_sales#104]
Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]

View file

@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
InputAdapter
Union
WholeStageCodegen (26)
Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #1
WholeStageCodegen (8)
@ -53,7 +53,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #14
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #2
WholeStageCodegen (25)
@ -165,10 +165,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (52)
Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #15
WholeStageCodegen (51)
@ -189,10 +189,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
InputAdapter
ReusedExchange [d_date_sk] #12
WholeStageCodegen (78)
Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #16
WholeStageCodegen (77)

View file

@ -496,15 +496,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43]
Results [7]: [store AS channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#45, count(1)#43 AS number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46]
(86) Filter [codegen id : 78]
Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47 as decimal(32,6)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
(87) Project [codegen id : 78]
Output [6]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46]
Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47]
Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46]
(88) ReusedExchange [Reuses operator id: 4]
Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4]
@ -584,18 +584,18 @@ Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61,
Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65]
Results [7]: [store AS channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#67, count(1)#65 AS number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69]
Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68]
(106) Filter [codegen id : 77]
Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#48, [id=#49] as decimal(32,6))))
Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
(107) Project [codegen id : 77]
Output [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69]
Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68]
(108) BroadcastExchange
Input [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70]
(109) BroadcastHashJoin [codegen id : 78]
@ -604,12 +604,12 @@ Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56]
Join condition: None
(110) TakeOrderedAndProject
Input [12]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
===== Subqueries =====
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#48, [id=#49]
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48]
* HashAggregate (136)
+- Exchange (135)
+- * HashAggregate (134)
@ -780,7 +780,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_d
Output [1]: [d_week_seq#29]
Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90]
Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#48, [id=#49]
Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48]
Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51]
* Project (144)

View file

@ -1,7 +1,7 @@
TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
WholeStageCodegen (78)
BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #2
WholeStageCodegen (8)
@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #16
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #1
WholeStageCodegen (38)
@ -190,10 +190,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
InputAdapter
BroadcastExchange #17
WholeStageCodegen (77)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #2
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #18
WholeStageCodegen (76)

View file

@ -446,15 +446,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40]
Results [7]: [store AS channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#42, count(1)#40 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43]
(76) Filter [codegen id : 52]
Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6))))
(77) Project [codegen id : 52]
Output [6]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43]
Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43]
(78) Scan parquet default.store_sales
Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4]
@ -537,18 +537,18 @@ Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58,
Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62]
Results [7]: [store AS channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#64, count(1)#62 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66]
Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65]
(96) Filter [codegen id : 51]
Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6))))
(97) Project [codegen id : 51]
Output [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66]
Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65]
(98) BroadcastExchange
Input [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67]
(99) BroadcastHashJoin [codegen id : 52]
@ -557,12 +557,12 @@ Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50]
Join condition: None
(100) TakeOrderedAndProject
Input [12]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Arguments: 100, [channel#41 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
===== Subqueries =====
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#45, [id=#46]
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45]
* HashAggregate (126)
+- Exchange (125)
+- * HashAggregate (124)
@ -733,7 +733,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_d
Output [1]: [d_week_seq#28]
Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87]
Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45]
Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52]
* Project (134)

View file

@ -1,7 +1,7 @@
TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
WholeStageCodegen (52)
BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #2
WholeStageCodegen (8)
@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #13
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #1
WholeStageCodegen (25)
@ -166,10 +166,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
InputAdapter
BroadcastExchange #14
WholeStageCodegen (51)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #2
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #15
WholeStageCodegen (50)

View file

@ -73,19 +73,19 @@ Input [2]: [i_manufact#2, count#9]
Keys [1]: [i_manufact#2]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#11]
Results [3]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13, true AS alwaysTrue#14]
Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13]
(12) Filter [codegen id : 2]
Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14]
Condition : (if (isnull(alwaysTrue#14)) 0 else item_cnt#12 > 0)
Input [2]: [item_cnt#12, i_manufact#2#13]
Condition : (item_cnt#12 > 0)
(13) Project [codegen id : 2]
Output [1]: [i_manufact#2#13]
Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14]
Input [2]: [item_cnt#12, i_manufact#2#13]
(14) BroadcastExchange
Input [1]: [i_manufact#2#13]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#15]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14]
(15) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [i_manufact#2]
@ -105,7 +105,7 @@ Results [1]: [i_product_name#3]
(18) Exchange
Input [1]: [i_product_name#3]
Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#16]
Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15]
(19) HashAggregate [codegen id : 4]
Input [1]: [i_product_name#3]

View file

@ -16,8 +16,8 @@ TakeOrderedAndProject [i_product_name]
BroadcastExchange #2
WholeStageCodegen (2)
Project [i_manufact]
Filter [alwaysTrue,item_cnt]
HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,alwaysTrue,count]
Filter [item_cnt]
HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count]
InputAdapter
Exchange [i_manufact] #3
WholeStageCodegen (1)

View file

@ -73,19 +73,19 @@ Input [2]: [i_manufact#2, count#9]
Keys [1]: [i_manufact#2]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#11]
Results [3]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13, true AS alwaysTrue#14]
Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13]
(12) Filter [codegen id : 2]
Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14]
Condition : (if (isnull(alwaysTrue#14)) 0 else item_cnt#12 > 0)
Input [2]: [item_cnt#12, i_manufact#2#13]
Condition : (item_cnt#12 > 0)
(13) Project [codegen id : 2]
Output [1]: [i_manufact#2#13]
Input [3]: [item_cnt#12, i_manufact#2#13, alwaysTrue#14]
Input [2]: [item_cnt#12, i_manufact#2#13]
(14) BroadcastExchange
Input [1]: [i_manufact#2#13]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#15]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14]
(15) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [i_manufact#2]
@ -105,7 +105,7 @@ Results [1]: [i_product_name#3]
(18) Exchange
Input [1]: [i_product_name#3]
Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#16]
Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15]
(19) HashAggregate [codegen id : 4]
Input [1]: [i_product_name#3]

View file

@ -16,8 +16,8 @@ TakeOrderedAndProject [i_product_name]
BroadcastExchange #2
WholeStageCodegen (2)
Project [i_manufact]
Filter [alwaysTrue,item_cnt]
HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,alwaysTrue,count]
Filter [item_cnt]
HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count]
InputAdapter
Exchange [i_manufact] #3
WholeStageCodegen (1)

View file

@ -496,15 +496,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43]
Results [7]: [store AS channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#45, count(1)#43 AS number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46]
(86) Filter [codegen id : 78]
Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47 as decimal(32,6)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
(87) Project [codegen id : 78]
Output [6]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46]
Input [7]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#47]
Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46]
(88) ReusedExchange [Reuses operator id: 4]
Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4]
@ -584,18 +584,18 @@ Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61,
Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65]
Results [7]: [store AS channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#67, count(1)#65 AS number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69]
Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68]
(106) Filter [codegen id : 77]
Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#48, [id=#49] as decimal(32,6))))
Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6))))
(107) Project [codegen id : 77]
Output [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Input [7]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#69]
Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68]
(108) BroadcastExchange
Input [6]: [channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70]
(109) BroadcastHashJoin [codegen id : 78]
@ -604,12 +604,12 @@ Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56]
Join condition: None
(110) TakeOrderedAndProject
Input [12]: [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#44, i_brand_id#7, i_class_id#8, i_category_id#9, sales#45, number_sales#46, channel#66, i_brand_id#54, i_class_id#55, i_category_id#56, sales#67, number_sales#68]
Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67]
===== Subqueries =====
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#48, [id=#49]
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48]
* HashAggregate (136)
+- Exchange (135)
+- * HashAggregate (134)
@ -780,7 +780,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_d
Output [1]: [d_week_seq#29]
Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90]
Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#48, [id=#49]
Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48]
Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51]
* Project (144)

View file

@ -1,7 +1,7 @@
TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
WholeStageCodegen (78)
BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #2
WholeStageCodegen (8)
@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #16
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #1
WholeStageCodegen (38)
@ -190,10 +190,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
InputAdapter
BroadcastExchange #17
WholeStageCodegen (77)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #2
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #18
WholeStageCodegen (76)

View file

@ -446,15 +446,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40]
Results [7]: [store AS channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#42, count(1)#40 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43]
(76) Filter [codegen id : 52]
Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6))))
(77) Project [codegen id : 52]
Output [6]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43]
Input [7]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43]
(78) Scan parquet default.store_sales
Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4]
@ -537,18 +537,18 @@ Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58,
Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62]
Results [7]: [store AS channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#64, count(1)#62 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66]
Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65]
(96) Filter [codegen id : 51]
Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6))))
(97) Project [codegen id : 51]
Output [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Input [7]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#66]
Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65]
(98) BroadcastExchange
Input [6]: [channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67]
(99) BroadcastHashJoin [codegen id : 52]
@ -557,12 +557,12 @@ Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50]
Join condition: None
(100) TakeOrderedAndProject
Input [12]: [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Arguments: 100, [channel#41 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#41, i_brand_id#6, i_class_id#7, i_category_id#8, sales#42, number_sales#43, channel#63, i_brand_id#48, i_class_id#49, i_category_id#50, sales#64, number_sales#65]
Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64]
===== Subqueries =====
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#45, [id=#46]
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45]
* HashAggregate (126)
+- Exchange (125)
+- * HashAggregate (124)
@ -733,7 +733,7 @@ Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_d
Output [1]: [d_week_seq#28]
Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87]
Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45]
Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52]
* Project (134)

View file

@ -1,7 +1,7 @@
TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
WholeStageCodegen (52)
BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #2
WholeStageCodegen (8)
@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #13
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #1
WholeStageCodegen (25)
@ -166,10 +166,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_
InputAdapter
BroadcastExchange #14
WholeStageCodegen (51)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #2
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #15
WholeStageCodegen (50)

View file

@ -608,15 +608,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41]
Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#43, count(1)#41 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
(86) Filter [codegen id : 39]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(87) Project [codegen id : 39]
Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45]
Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44]
(88) Scan parquet default.catalog_sales
Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49]
@ -692,15 +692,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59]
Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#61, count(1)#59 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62]
(105) Filter [codegen id : 78]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(106) Project [codegen id : 78]
Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63]
Output [6]: [catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62]
(107) Scan parquet default.web_sales
Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65]
@ -776,35 +776,35 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75]
Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#77, count(1)#75 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78]
(124) Filter [codegen id : 117]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(125) Project [codegen id : 117]
Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#79]
Output [6]: [web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78]
(126) Union
(127) HashAggregate [codegen id : 118]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)]
Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82]
Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85]
Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85]
(128) Exchange
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#86]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#86]
(129) HashAggregate [codegen id : 119]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#43), sum(number_sales#44)]
Aggregate Attributes [2]: [sum(sales#43)#87, sum(number_sales#44)#88]
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#43)#87 AS sum_sales#89, sum(number_sales#44)#88 AS number_sales#90]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#42), sum(number_sales#43)]
Aggregate Attributes [2]: [sum(sales#42)#87, sum(number_sales#43)#88]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#42)#87 AS sum_sales#89, sum(number_sales#43)#88 AS number_sales#90]
(130) ReusedExchange [Reuses operator id: 84]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93]
@ -814,15 +814,15 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94, count(1)#95]
Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#43, count(1)#95 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#42, count(1)#95 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96]
(132) Filter [codegen id : 158]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(133) Project [codegen id : 158]
Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96]
Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96]
(134) ReusedExchange [Reuses operator id: 103]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99]
@ -832,435 +832,435 @@ Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, cou
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100, count(1)#101]
Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#61, count(1)#101 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#60, count(1)#101 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102]
(136) Filter [codegen id : 197]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(137) Project [codegen id : 197]
Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102]
Output [6]: [catalog AS channel#103, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102]
(138) ReusedExchange [Reuses operator id: 122]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#103, isEmpty#104, count#105]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106]
(139) HashAggregate [codegen id : 236]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#103, isEmpty#104, count#105]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#106, count(1)#107]
Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#106 AS sales#77, count(1)#107 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#106 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#76, count(1)#108 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109]
(140) Filter [codegen id : 236]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(141) Project [codegen id : 236]
Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#108]
Output [6]: [web AS channel#110, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109]
(142) Union
(143) HashAggregate [codegen id : 237]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)]
Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111]
Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#112, isEmpty#113, sum#114]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)]
Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113]
Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116]
(144) Exchange
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#112, isEmpty#113, sum#114]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#115]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#117]
(145) HashAggregate [codegen id : 238]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#112, isEmpty#113, sum#114]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#43), sum(number_sales#44)]
Aggregate Attributes [2]: [sum(sales#43)#116, sum(number_sales#44)#117]
Results [5]: [channel#42, i_brand_id#7, i_class_id#8, sum(sales#43)#116 AS sum_sales#89, sum(number_sales#44)#117 AS number_sales#90]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#42), sum(number_sales#43)]
Aggregate Attributes [2]: [sum(sales#42)#118, sum(number_sales#43)#119]
Results [5]: [channel#47, i_brand_id#7, i_class_id#8, sum(sales#42)#118 AS sum_sales#89, sum(number_sales#43)#119 AS number_sales#90]
(146) HashAggregate [codegen id : 238]
Input [5]: [channel#42, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90]
Keys [3]: [channel#42, i_brand_id#7, i_class_id#8]
Input [5]: [channel#47, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90]
Keys [3]: [channel#47, i_brand_id#7, i_class_id#8]
Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)]
Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120]
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, sum#121, isEmpty#122, sum#123]
Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125]
(147) Exchange
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, sum#121, isEmpty#122, sum#123]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, 5), true, [id=#124]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, 5), true, [id=#126]
(148) HashAggregate [codegen id : 239]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, sum#121, isEmpty#122, sum#123]
Keys [3]: [channel#42, i_brand_id#7, i_class_id#8]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125]
Keys [3]: [channel#47, i_brand_id#7, i_class_id#8]
Functions [2]: [sum(sum_sales#89), sum(number_sales#90)]
Aggregate Attributes [2]: [sum(sum_sales#89)#125, sum(number_sales#90)#126]
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, null AS i_category_id#127, sum(sum_sales#89)#125 AS sum(sum_sales)#128, sum(number_sales#90)#126 AS sum(number_sales)#129]
Aggregate Attributes [2]: [sum(sum_sales#89)#127, sum(number_sales#90)#128]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, null AS i_category_id#129, sum(sum_sales#89)#127 AS sum(sum_sales)#130, sum(number_sales#90)#128 AS sum(number_sales)#131]
(149) Union
(150) HashAggregate [codegen id : 240]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(151) Exchange
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#130]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#132]
(152) HashAggregate [codegen id : 241]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(153) ReusedExchange [Reuses operator id: 84]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#131, isEmpty#132, count#133]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135]
(154) HashAggregate [codegen id : 280]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#131, isEmpty#132, count#133]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#134, count(1)#135]
Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#134 AS sales#43, count(1)#135 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#134 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136, count(1)#137]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sales#42, count(1)#137 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138]
(155) Filter [codegen id : 280]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(156) Project [codegen id : 280]
Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136]
Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138]
(157) ReusedExchange [Reuses operator id: 103]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#137, isEmpty#138, count#139]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141]
(158) HashAggregate [codegen id : 319]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#137, isEmpty#138, count#139]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#140, count(1)#141]
Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#140 AS sales#61, count(1)#141 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#140 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142, count(1)#143]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sales#60, count(1)#143 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144]
(159) Filter [codegen id : 319]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(160) Project [codegen id : 319]
Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142]
Output [6]: [catalog AS channel#145, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144]
(161) ReusedExchange [Reuses operator id: 122]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#143, isEmpty#144, count#145]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148]
(162) HashAggregate [codegen id : 358]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#143, isEmpty#144, count#145]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#146, count(1)#147]
Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#146 AS sales#77, count(1)#147 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#146 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#76, count(1)#150 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151]
(163) Filter [codegen id : 358]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(164) Project [codegen id : 358]
Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148]
Output [6]: [web AS channel#152, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151]
(165) Union
(166) HashAggregate [codegen id : 359]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)]
Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151]
Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)]
Aggregate Attributes [3]: [sum#153, isEmpty#154, sum#155]
Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158]
(167) Exchange
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#155]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#159]
(168) HashAggregate [codegen id : 360]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#43), sum(number_sales#44)]
Aggregate Attributes [2]: [sum(sales#43)#156, sum(number_sales#44)#157]
Results [4]: [channel#42, i_brand_id#7, sum(sales#43)#156 AS sum_sales#89, sum(number_sales#44)#157 AS number_sales#90]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#42), sum(number_sales#43)]
Aggregate Attributes [2]: [sum(sales#42)#160, sum(number_sales#43)#161]
Results [4]: [channel#47, i_brand_id#7, sum(sales#42)#160 AS sum_sales#89, sum(number_sales#43)#161 AS number_sales#90]
(169) HashAggregate [codegen id : 360]
Input [4]: [channel#42, i_brand_id#7, sum_sales#89, number_sales#90]
Keys [2]: [channel#42, i_brand_id#7]
Input [4]: [channel#47, i_brand_id#7, sum_sales#89, number_sales#90]
Keys [2]: [channel#47, i_brand_id#7]
Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)]
Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160]
Results [5]: [channel#42, i_brand_id#7, sum#161, isEmpty#162, sum#163]
Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164]
Results [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167]
(170) Exchange
Input [5]: [channel#42, i_brand_id#7, sum#161, isEmpty#162, sum#163]
Arguments: hashpartitioning(channel#42, i_brand_id#7, 5), true, [id=#164]
Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167]
Arguments: hashpartitioning(channel#47, i_brand_id#7, 5), true, [id=#168]
(171) HashAggregate [codegen id : 361]
Input [5]: [channel#42, i_brand_id#7, sum#161, isEmpty#162, sum#163]
Keys [2]: [channel#42, i_brand_id#7]
Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167]
Keys [2]: [channel#47, i_brand_id#7]
Functions [2]: [sum(sum_sales#89), sum(number_sales#90)]
Aggregate Attributes [2]: [sum(sum_sales#89)#165, sum(number_sales#90)#166]
Results [6]: [channel#42, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#89)#165 AS sum(sum_sales)#169, sum(number_sales#90)#166 AS sum(number_sales)#170]
Aggregate Attributes [2]: [sum(sum_sales#89)#169, sum(number_sales#90)#170]
Results [6]: [channel#47, i_brand_id#7, null AS i_class_id#171, null AS i_category_id#172, sum(sum_sales#89)#169 AS sum(sum_sales)#173, sum(number_sales#90)#170 AS sum(number_sales)#174]
(172) Union
(173) HashAggregate [codegen id : 362]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(174) Exchange
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#171]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#175]
(175) HashAggregate [codegen id : 363]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(176) ReusedExchange [Reuses operator id: 84]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#172, isEmpty#173, count#174]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178]
(177) HashAggregate [codegen id : 402]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#172, isEmpty#173, count#174]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#175, count(1)#176]
Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sales#43, count(1)#176 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179, count(1)#180]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sales#42, count(1)#180 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181]
(178) Filter [codegen id : 402]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(179) Project [codegen id : 402]
Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177]
Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181]
(180) ReusedExchange [Reuses operator id: 103]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#178, isEmpty#179, count#180]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184]
(181) HashAggregate [codegen id : 441]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#178, isEmpty#179, count#180]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182]
Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#61, count(1)#182 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185, count(1)#186]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sales#60, count(1)#186 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187]
(182) Filter [codegen id : 441]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(183) Project [codegen id : 441]
Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183]
Output [6]: [catalog AS channel#188, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187]
(184) ReusedExchange [Reuses operator id: 122]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191]
(185) HashAggregate [codegen id : 480]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188]
Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#77, count(1)#188 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#76, count(1)#193 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194]
(186) Filter [codegen id : 480]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(187) Project [codegen id : 480]
Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#189]
Output [6]: [web AS channel#195, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194]
(188) Union
(189) HashAggregate [codegen id : 481]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)]
Aggregate Attributes [3]: [sum#190, isEmpty#191, sum#192]
Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#193, isEmpty#194, sum#195]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)]
Aggregate Attributes [3]: [sum#196, isEmpty#197, sum#198]
Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201]
(190) Exchange
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#193, isEmpty#194, sum#195]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#196]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#202]
(191) HashAggregate [codegen id : 482]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#193, isEmpty#194, sum#195]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#43), sum(number_sales#44)]
Aggregate Attributes [2]: [sum(sales#43)#197, sum(number_sales#44)#198]
Results [3]: [channel#42, sum(sales#43)#197 AS sum_sales#89, sum(number_sales#44)#198 AS number_sales#90]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#42), sum(number_sales#43)]
Aggregate Attributes [2]: [sum(sales#42)#203, sum(number_sales#43)#204]
Results [3]: [channel#47, sum(sales#42)#203 AS sum_sales#89, sum(number_sales#43)#204 AS number_sales#90]
(192) HashAggregate [codegen id : 482]
Input [3]: [channel#42, sum_sales#89, number_sales#90]
Keys [1]: [channel#42]
Input [3]: [channel#47, sum_sales#89, number_sales#90]
Keys [1]: [channel#47]
Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)]
Aggregate Attributes [3]: [sum#199, isEmpty#200, sum#201]
Results [4]: [channel#42, sum#202, isEmpty#203, sum#204]
Aggregate Attributes [3]: [sum#205, isEmpty#206, sum#207]
Results [4]: [channel#47, sum#208, isEmpty#209, sum#210]
(193) Exchange
Input [4]: [channel#42, sum#202, isEmpty#203, sum#204]
Arguments: hashpartitioning(channel#42, 5), true, [id=#205]
Input [4]: [channel#47, sum#208, isEmpty#209, sum#210]
Arguments: hashpartitioning(channel#47, 5), true, [id=#211]
(194) HashAggregate [codegen id : 483]
Input [4]: [channel#42, sum#202, isEmpty#203, sum#204]
Keys [1]: [channel#42]
Input [4]: [channel#47, sum#208, isEmpty#209, sum#210]
Keys [1]: [channel#47]
Functions [2]: [sum(sum_sales#89), sum(number_sales#90)]
Aggregate Attributes [2]: [sum(sum_sales#89)#206, sum(number_sales#90)#207]
Results [6]: [channel#42, null AS i_brand_id#208, null AS i_class_id#209, null AS i_category_id#210, sum(sum_sales#89)#206 AS sum(sum_sales)#211, sum(number_sales#90)#207 AS sum(number_sales)#212]
Aggregate Attributes [2]: [sum(sum_sales#89)#212, sum(number_sales#90)#213]
Results [6]: [channel#47, null AS i_brand_id#214, null AS i_class_id#215, null AS i_category_id#216, sum(sum_sales#89)#212 AS sum(sum_sales)#217, sum(number_sales#90)#213 AS sum(number_sales)#218]
(195) Union
(196) HashAggregate [codegen id : 484]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(197) Exchange
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#213]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#219]
(198) HashAggregate [codegen id : 485]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(199) ReusedExchange [Reuses operator id: 84]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216]
(200) HashAggregate [codegen id : 524]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218]
Results [7]: [store AS channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#43, count(1)#218 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219]
(201) Filter [codegen id : 524]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
(202) Project [codegen id : 524]
Output [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#219]
(203) ReusedExchange [Reuses operator id: 103]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222]
(204) HashAggregate [codegen id : 563]
(200) HashAggregate [codegen id : 524]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224]
Results [7]: [catalog AS channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#61, count(1)#224 AS number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#42, count(1)#224 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225]
(205) Filter [codegen id : 563]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
(201) Filter [codegen id : 524]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(206) Project [codegen id : 563]
Output [6]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62]
Input [7]: [channel#60, i_brand_id#7, i_class_id#8, i_category_id#9, sales#61, number_sales#62, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#225]
(202) Project [codegen id : 524]
Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225]
(207) ReusedExchange [Reuses operator id: 122]
(203) ReusedExchange [Reuses operator id: 103]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228]
(208) HashAggregate [codegen id : 602]
(204) HashAggregate [codegen id : 563]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229, count(1)#230]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sales#60, count(1)#230 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231]
(205) Filter [codegen id : 563]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(206) Project [codegen id : 563]
Output [6]: [catalog AS channel#232, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231]
(207) ReusedExchange [Reuses operator id: 122]
Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235]
(208) HashAggregate [codegen id : 602]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235]
Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#229, count(1)#230]
Results [7]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sales#77, count(1)#230 AS number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237]
Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#76, count(1)#237 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238]
(209) Filter [codegen id : 602]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6))))
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6))))
(210) Project [codegen id : 602]
Output [6]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78]
Input [7]: [channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#77, number_sales#78, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#231]
Output [6]: [web AS channel#239, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77]
Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238]
(211) Union
(212) HashAggregate [codegen id : 603]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#43), partial_sum(number_sales#44)]
Aggregate Attributes [3]: [sum#232, isEmpty#233, sum#234]
Results [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#235, isEmpty#236, sum#237]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)]
Aggregate Attributes [3]: [sum#240, isEmpty#241, sum#242]
Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245]
(213) Exchange
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#235, isEmpty#236, sum#237]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#238]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#246]
(214) HashAggregate [codegen id : 604]
Input [7]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum#235, isEmpty#236, sum#237]
Keys [4]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#43), sum(number_sales#44)]
Aggregate Attributes [2]: [sum(sales#43)#239, sum(number_sales#44)#240]
Results [2]: [sum(sales#43)#239 AS sum_sales#89, sum(number_sales#44)#240 AS number_sales#90]
Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245]
Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9]
Functions [2]: [sum(sales#42), sum(number_sales#43)]
Aggregate Attributes [2]: [sum(sales#42)#247, sum(number_sales#43)#248]
Results [2]: [sum(sales#42)#247 AS sum_sales#89, sum(number_sales#43)#248 AS number_sales#90]
(215) HashAggregate [codegen id : 604]
Input [2]: [sum_sales#89, number_sales#90]
Keys: []
Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)]
Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243]
Results [3]: [sum#244, isEmpty#245, sum#246]
Aggregate Attributes [3]: [sum#249, isEmpty#250, sum#251]
Results [3]: [sum#252, isEmpty#253, sum#254]
(216) Exchange
Input [3]: [sum#244, isEmpty#245, sum#246]
Arguments: SinglePartition, true, [id=#247]
Input [3]: [sum#252, isEmpty#253, sum#254]
Arguments: SinglePartition, true, [id=#255]
(217) HashAggregate [codegen id : 605]
Input [3]: [sum#244, isEmpty#245, sum#246]
Input [3]: [sum#252, isEmpty#253, sum#254]
Keys: []
Functions [2]: [sum(sum_sales#89), sum(number_sales#90)]
Aggregate Attributes [2]: [sum(sum_sales#89)#248, sum(number_sales#90)#249]
Results [6]: [null AS channel#250, null AS i_brand_id#251, null AS i_class_id#252, null AS i_category_id#253, sum(sum_sales#89)#248 AS sum(sum_sales)#254, sum(number_sales#90)#249 AS sum(number_sales)#255]
Aggregate Attributes [2]: [sum(sum_sales#89)#256, sum(number_sales#90)#257]
Results [6]: [null AS channel#258, null AS i_brand_id#259, null AS i_class_id#260, null AS i_category_id#261, sum(sum_sales#89)#256 AS sum(sum_sales)#262, sum(number_sales#90)#257 AS sum(number_sales)#263]
(218) Union
(219) HashAggregate [codegen id : 606]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(220) Exchange
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#256]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#264]
(221) HashAggregate [codegen id : 607]
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
(222) TakeOrderedAndProject
Input [6]: [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: 100, [channel#42 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#42, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
Arguments: 100, [channel#47 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90]
===== Subqueries =====
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#46, [id=#47]
Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46]
* HashAggregate (252)
+- Exchange (251)
+- * HashAggregate (250)
@ -1327,7 +1327,7 @@ Input [2]: [d_date_sk#10, d_year#11]
(230) BroadcastExchange
Input [1]: [d_date_sk#10]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#257]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#265]
(231) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#1]
@ -1335,7 +1335,7 @@ Right keys [1]: [d_date_sk#10]
Join condition: None
(232) Project [codegen id : 2]
Output [2]: [ss_quantity#3 AS quantity#258, ss_list_price#4 AS list_price#259]
Output [2]: [ss_quantity#3 AS quantity#266, ss_list_price#4 AS list_price#267]
Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10]
(233) Scan parquet default.catalog_sales
@ -1372,7 +1372,7 @@ Input [2]: [d_date_sk#10, d_year#11]
(240) BroadcastExchange
Input [1]: [d_date_sk#10]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#268]
(241) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#18]
@ -1380,7 +1380,7 @@ Right keys [1]: [d_date_sk#10]
Join condition: None
(242) Project [codegen id : 4]
Output [2]: [cs_quantity#48 AS quantity#261, cs_list_price#49 AS list_price#262]
Output [2]: [cs_quantity#48 AS quantity#269, cs_list_price#49 AS list_price#270]
Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10]
(243) Scan parquet default.web_sales
@ -1406,55 +1406,55 @@ Right keys [1]: [d_date_sk#10]
Join condition: None
(248) Project [codegen id : 6]
Output [2]: [ws_quantity#64 AS quantity#263, ws_list_price#65 AS list_price#264]
Output [2]: [ws_quantity#64 AS quantity#271, ws_list_price#65 AS list_price#272]
Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10]
(249) Union
(250) HashAggregate [codegen id : 7]
Input [2]: [quantity#258, list_price#259]
Input [2]: [quantity#266, list_price#267]
Keys: []
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#265, count#266]
Results [2]: [sum#267, count#268]
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#273, count#274]
Results [2]: [sum#275, count#276]
(251) Exchange
Input [2]: [sum#267, count#268]
Arguments: SinglePartition, true, [id=#269]
Input [2]: [sum#275, count#276]
Arguments: SinglePartition, true, [id=#277]
(252) HashAggregate [codegen id : 8]
Input [2]: [sum#267, count#268]
Input [2]: [sum#275, count#276]
Keys: []
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))#270]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))#270 AS average_sales#271]
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278 AS average_sales#279]
Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:7 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:7 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:8 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:8 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:9 Hosting operator id = 163 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:9 Hosting operator id = 163 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:10 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:10 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:11 Hosting operator id = 182 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:11 Hosting operator id = 182 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:12 Hosting operator id = 186 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:12 Hosting operator id = 186 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:13 Hosting operator id = 201 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:13 Hosting operator id = 201 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:14 Hosting operator id = 205 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:14 Hosting operator id = 205 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]
Subquery:15 Hosting operator id = 209 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47]
Subquery:15 Hosting operator id = 209 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46]

View file

@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (39)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #1
WholeStageCodegen (8)
@ -90,7 +90,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #22
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),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #6
WholeStageCodegen (38)
@ -226,10 +226,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
ReusedExchange [ss_item_sk] #8
WholeStageCodegen (78)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #23
WholeStageCodegen (77)
@ -258,10 +258,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #18
WholeStageCodegen (117)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #25
WholeStageCodegen (116)
@ -303,24 +303,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (158)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (197)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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 (236)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #25
WholeStageCodegen (361)
@ -337,24 +337,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (280)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (319)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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 (358)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #25
WholeStageCodegen (483)
@ -371,24 +371,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (402)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (441)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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 (480)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #25
WholeStageCodegen (605)
@ -405,23 +405,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (524)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (563)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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 (602)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #25

View file

@ -552,15 +552,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38]
Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#40, count(1)#38 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41]
(76) Filter [codegen id : 26]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 as decimal(32,6)) > cast(Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(77) Project [codegen id : 26]
Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42]
Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41]
(78) Scan parquet default.catalog_sales
Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46]
@ -624,15 +624,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55]
Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#57, count(1)#55 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58]
(92) Filter [codegen id : 52]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(93) Project [codegen id : 52]
Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#59]
Output [6]: [catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58]
(94) Scan parquet default.web_sales
Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61]
@ -696,35 +696,35 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70]
Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#72, count(1)#70 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73]
(108) Filter [codegen id : 78]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(109) Project [codegen id : 78]
Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#74]
Output [6]: [web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73]
(110) Union
(111) HashAggregate [codegen id : 79]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)]
Aggregate Attributes [3]: [sum#75, isEmpty#76, sum#77]
Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80]
Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80]
(112) Exchange
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#81]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#81]
(113) HashAggregate [codegen id : 80]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#40), sum(number_sales#41)]
Aggregate Attributes [2]: [sum(sales#40)#82, sum(number_sales#41)#83]
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#40)#82 AS sum_sales#84, sum(number_sales#41)#83 AS number_sales#85]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#39), sum(number_sales#40)]
Aggregate Attributes [2]: [sum(sales#39)#82, sum(number_sales#40)#83]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#39)#82 AS sum_sales#84, sum(number_sales#40)#83 AS number_sales#85]
(114) ReusedExchange [Reuses operator id: 74]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88]
@ -734,15 +734,15 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89, count(1)#90]
Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#40, count(1)#90 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#39, count(1)#90 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91]
(116) Filter [codegen id : 106]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(117) Project [codegen id : 106]
Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91]
Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91]
(118) ReusedExchange [Reuses operator id: 90]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94]
@ -752,435 +752,435 @@ Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, cou
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95, count(1)#96]
Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#57, count(1)#96 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#56, count(1)#96 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97]
(120) Filter [codegen id : 132]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(121) Project [codegen id : 132]
Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97]
Output [6]: [catalog AS channel#98, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97]
(122) ReusedExchange [Reuses operator id: 106]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#98, isEmpty#99, count#100]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101]
(123) HashAggregate [codegen id : 158]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#98, isEmpty#99, count#100]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#101, count(1)#102]
Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#101 AS sales#72, count(1)#102 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#101 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#71, count(1)#103 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104]
(124) Filter [codegen id : 158]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(125) Project [codegen id : 158]
Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#103]
Output [6]: [web AS channel#105, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104]
(126) Union
(127) HashAggregate [codegen id : 159]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)]
Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106]
Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#107, isEmpty#108, sum#109]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)]
Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108]
Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111]
(128) Exchange
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#107, isEmpty#108, sum#109]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#110]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#112]
(129) HashAggregate [codegen id : 160]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#107, isEmpty#108, sum#109]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#40), sum(number_sales#41)]
Aggregate Attributes [2]: [sum(sales#40)#111, sum(number_sales#41)#112]
Results [5]: [channel#39, i_brand_id#6, i_class_id#7, sum(sales#40)#111 AS sum_sales#84, sum(number_sales#41)#112 AS number_sales#85]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#39), sum(number_sales#40)]
Aggregate Attributes [2]: [sum(sales#39)#113, sum(number_sales#40)#114]
Results [5]: [channel#44, i_brand_id#6, i_class_id#7, sum(sales#39)#113 AS sum_sales#84, sum(number_sales#40)#114 AS number_sales#85]
(130) HashAggregate [codegen id : 160]
Input [5]: [channel#39, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85]
Keys [3]: [channel#39, i_brand_id#6, i_class_id#7]
Input [5]: [channel#44, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85]
Keys [3]: [channel#44, i_brand_id#6, i_class_id#7]
Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)]
Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115]
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, sum#116, isEmpty#117, sum#118]
Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120]
(131) Exchange
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, sum#116, isEmpty#117, sum#118]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, 5), true, [id=#119]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, 5), true, [id=#121]
(132) HashAggregate [codegen id : 161]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, sum#116, isEmpty#117, sum#118]
Keys [3]: [channel#39, i_brand_id#6, i_class_id#7]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120]
Keys [3]: [channel#44, i_brand_id#6, i_class_id#7]
Functions [2]: [sum(sum_sales#84), sum(number_sales#85)]
Aggregate Attributes [2]: [sum(sum_sales#84)#120, sum(number_sales#85)#121]
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, null AS i_category_id#122, sum(sum_sales#84)#120 AS sum(sum_sales)#123, sum(number_sales#85)#121 AS sum(number_sales)#124]
Aggregate Attributes [2]: [sum(sum_sales#84)#122, sum(number_sales#85)#123]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, null AS i_category_id#124, sum(sum_sales#84)#122 AS sum(sum_sales)#125, sum(number_sales#85)#123 AS sum(number_sales)#126]
(133) Union
(134) HashAggregate [codegen id : 162]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(135) Exchange
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#125]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#127]
(136) HashAggregate [codegen id : 163]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(137) ReusedExchange [Reuses operator id: 74]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#126, isEmpty#127, count#128]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130]
(138) HashAggregate [codegen id : 189]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#126, isEmpty#127, count#128]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#129, count(1)#130]
Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#129 AS sales#40, count(1)#130 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131, count(1)#132]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sales#39, count(1)#132 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133]
(139) Filter [codegen id : 189]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(140) Project [codegen id : 189]
Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131]
Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133]
(141) ReusedExchange [Reuses operator id: 90]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#132, isEmpty#133, count#134]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136]
(142) HashAggregate [codegen id : 215]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#132, isEmpty#133, count#134]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#135, count(1)#136]
Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sales#57, count(1)#136 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137, count(1)#138]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sales#56, count(1)#138 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139]
(143) Filter [codegen id : 215]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(144) Project [codegen id : 215]
Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137]
Output [6]: [catalog AS channel#140, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139]
(145) ReusedExchange [Reuses operator id: 106]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#138, isEmpty#139, count#140]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143]
(146) HashAggregate [codegen id : 241]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#138, isEmpty#139, count#140]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#141, count(1)#142]
Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sales#72, count(1)#142 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144, count(1)#145]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sales#71, count(1)#145 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146]
(147) Filter [codegen id : 241]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(148) Project [codegen id : 241]
Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143]
Output [6]: [web AS channel#147, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146]
(149) Union
(150) HashAggregate [codegen id : 242]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)]
Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146]
Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#147, isEmpty#148, sum#149]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)]
Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150]
Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153]
(151) Exchange
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#147, isEmpty#148, sum#149]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#150]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#154]
(152) HashAggregate [codegen id : 243]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#147, isEmpty#148, sum#149]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#40), sum(number_sales#41)]
Aggregate Attributes [2]: [sum(sales#40)#151, sum(number_sales#41)#152]
Results [4]: [channel#39, i_brand_id#6, sum(sales#40)#151 AS sum_sales#84, sum(number_sales#41)#152 AS number_sales#85]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#39), sum(number_sales#40)]
Aggregate Attributes [2]: [sum(sales#39)#155, sum(number_sales#40)#156]
Results [4]: [channel#44, i_brand_id#6, sum(sales#39)#155 AS sum_sales#84, sum(number_sales#40)#156 AS number_sales#85]
(153) HashAggregate [codegen id : 243]
Input [4]: [channel#39, i_brand_id#6, sum_sales#84, number_sales#85]
Keys [2]: [channel#39, i_brand_id#6]
Input [4]: [channel#44, i_brand_id#6, sum_sales#84, number_sales#85]
Keys [2]: [channel#44, i_brand_id#6]
Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)]
Aggregate Attributes [3]: [sum#153, isEmpty#154, sum#155]
Results [5]: [channel#39, i_brand_id#6, sum#156, isEmpty#157, sum#158]
Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159]
Results [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162]
(154) Exchange
Input [5]: [channel#39, i_brand_id#6, sum#156, isEmpty#157, sum#158]
Arguments: hashpartitioning(channel#39, i_brand_id#6, 5), true, [id=#159]
Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162]
Arguments: hashpartitioning(channel#44, i_brand_id#6, 5), true, [id=#163]
(155) HashAggregate [codegen id : 244]
Input [5]: [channel#39, i_brand_id#6, sum#156, isEmpty#157, sum#158]
Keys [2]: [channel#39, i_brand_id#6]
Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162]
Keys [2]: [channel#44, i_brand_id#6]
Functions [2]: [sum(sum_sales#84), sum(number_sales#85)]
Aggregate Attributes [2]: [sum(sum_sales#84)#160, sum(number_sales#85)#161]
Results [6]: [channel#39, i_brand_id#6, null AS i_class_id#162, null AS i_category_id#163, sum(sum_sales#84)#160 AS sum(sum_sales)#164, sum(number_sales#85)#161 AS sum(number_sales)#165]
Aggregate Attributes [2]: [sum(sum_sales#84)#164, sum(number_sales#85)#165]
Results [6]: [channel#44, i_brand_id#6, null AS i_class_id#166, null AS i_category_id#167, sum(sum_sales#84)#164 AS sum(sum_sales)#168, sum(number_sales#85)#165 AS sum(number_sales)#169]
(156) Union
(157) HashAggregate [codegen id : 245]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(158) Exchange
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#166]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#170]
(159) HashAggregate [codegen id : 246]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(160) ReusedExchange [Reuses operator id: 74]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#167, isEmpty#168, count#169]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173]
(161) HashAggregate [codegen id : 272]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#167, isEmpty#168, count#169]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#170, count(1)#171]
Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#170 AS sales#40, count(1)#171 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#170 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#39, count(1)#175 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176]
(162) Filter [codegen id : 272]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(163) Project [codegen id : 272]
Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172]
Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176]
(164) ReusedExchange [Reuses operator id: 90]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#173, isEmpty#174, count#175]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179]
(165) HashAggregate [codegen id : 298]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#173, isEmpty#174, count#175]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#176, count(1)#177]
Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#176 AS sales#57, count(1)#177 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#176 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#56, count(1)#181 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182]
(166) Filter [codegen id : 298]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(167) Project [codegen id : 298]
Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178]
Output [6]: [catalog AS channel#183, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182]
(168) ReusedExchange [Reuses operator id: 106]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#179, isEmpty#180, count#181]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186]
(169) HashAggregate [codegen id : 324]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#179, isEmpty#180, count#181]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#182, count(1)#183]
Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#182 AS sales#72, count(1)#183 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#182 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#71, count(1)#188 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189]
(170) Filter [codegen id : 324]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(171) Project [codegen id : 324]
Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#184]
Output [6]: [web AS channel#190, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189]
(172) Union
(173) HashAggregate [codegen id : 325]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)]
Aggregate Attributes [3]: [sum#185, isEmpty#186, sum#187]
Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#188, isEmpty#189, sum#190]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)]
Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193]
Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196]
(174) Exchange
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#188, isEmpty#189, sum#190]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#191]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#197]
(175) HashAggregate [codegen id : 326]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#188, isEmpty#189, sum#190]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#40), sum(number_sales#41)]
Aggregate Attributes [2]: [sum(sales#40)#192, sum(number_sales#41)#193]
Results [3]: [channel#39, sum(sales#40)#192 AS sum_sales#84, sum(number_sales#41)#193 AS number_sales#85]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#39), sum(number_sales#40)]
Aggregate Attributes [2]: [sum(sales#39)#198, sum(number_sales#40)#199]
Results [3]: [channel#44, sum(sales#39)#198 AS sum_sales#84, sum(number_sales#40)#199 AS number_sales#85]
(176) HashAggregate [codegen id : 326]
Input [3]: [channel#39, sum_sales#84, number_sales#85]
Keys [1]: [channel#39]
Input [3]: [channel#44, sum_sales#84, number_sales#85]
Keys [1]: [channel#44]
Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)]
Aggregate Attributes [3]: [sum#194, isEmpty#195, sum#196]
Results [4]: [channel#39, sum#197, isEmpty#198, sum#199]
Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202]
Results [4]: [channel#44, sum#203, isEmpty#204, sum#205]
(177) Exchange
Input [4]: [channel#39, sum#197, isEmpty#198, sum#199]
Arguments: hashpartitioning(channel#39, 5), true, [id=#200]
Input [4]: [channel#44, sum#203, isEmpty#204, sum#205]
Arguments: hashpartitioning(channel#44, 5), true, [id=#206]
(178) HashAggregate [codegen id : 327]
Input [4]: [channel#39, sum#197, isEmpty#198, sum#199]
Keys [1]: [channel#39]
Input [4]: [channel#44, sum#203, isEmpty#204, sum#205]
Keys [1]: [channel#44]
Functions [2]: [sum(sum_sales#84), sum(number_sales#85)]
Aggregate Attributes [2]: [sum(sum_sales#84)#201, sum(number_sales#85)#202]
Results [6]: [channel#39, null AS i_brand_id#203, null AS i_class_id#204, null AS i_category_id#205, sum(sum_sales#84)#201 AS sum(sum_sales)#206, sum(number_sales#85)#202 AS sum(number_sales)#207]
Aggregate Attributes [2]: [sum(sum_sales#84)#207, sum(number_sales#85)#208]
Results [6]: [channel#44, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#84)#207 AS sum(sum_sales)#212, sum(number_sales#85)#208 AS sum(number_sales)#213]
(179) Union
(180) HashAggregate [codegen id : 328]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(181) Exchange
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#208]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#214]
(182) HashAggregate [codegen id : 329]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(183) ReusedExchange [Reuses operator id: 74]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#209, isEmpty#210, count#211]
(184) HashAggregate [codegen id : 355]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#209, isEmpty#210, count#211]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#212, count(1)#213]
Results [7]: [store AS channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#212 AS sales#40, count(1)#213 AS number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#212 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214]
(185) Filter [codegen id : 355]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
(186) Project [codegen id : 355]
Output [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#214]
(187) ReusedExchange [Reuses operator id: 90]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217]
(188) HashAggregate [codegen id : 381]
(184) HashAggregate [codegen id : 355]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#218, count(1)#219]
Results [7]: [catalog AS channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sales#57, count(1)#219 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218, count(1)#219]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sales#39, count(1)#219 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220]
(189) Filter [codegen id : 381]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
(185) Filter [codegen id : 355]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(190) Project [codegen id : 381]
Output [6]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58]
Input [7]: [channel#56, i_brand_id#6, i_class_id#7, i_category_id#8, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#220]
(186) Project [codegen id : 355]
Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220]
(191) ReusedExchange [Reuses operator id: 106]
(187) ReusedExchange [Reuses operator id: 90]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223]
(192) HashAggregate [codegen id : 407]
(188) HashAggregate [codegen id : 381]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#56, count(1)#225 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226]
(189) Filter [codegen id : 381]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(190) Project [codegen id : 381]
Output [6]: [catalog AS channel#227, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226]
(191) ReusedExchange [Reuses operator id: 106]
Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230]
(192) HashAggregate [codegen id : 407]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230]
Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225]
Results [7]: [web AS channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#72, count(1)#225 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231, count(1)#232]
Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sales#71, count(1)#232 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233]
(193) Filter [codegen id : 407]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#43, [id=#44] as decimal(32,6))))
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6))))
(194) Project [codegen id : 407]
Output [6]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73]
Input [7]: [channel#71, i_brand_id#6, i_class_id#7, i_category_id#8, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#226]
Output [6]: [web AS channel#234, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72]
Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233]
(195) Union
(196) HashAggregate [codegen id : 408]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sales#40, number_sales#41]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#40), partial_sum(number_sales#41)]
Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229]
Results [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#230, isEmpty#231, sum#232]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)]
Aggregate Attributes [3]: [sum#235, isEmpty#236, sum#237]
Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240]
(197) Exchange
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#230, isEmpty#231, sum#232]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#233]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#241]
(198) HashAggregate [codegen id : 409]
Input [7]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum#230, isEmpty#231, sum#232]
Keys [4]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#40), sum(number_sales#41)]
Aggregate Attributes [2]: [sum(sales#40)#234, sum(number_sales#41)#235]
Results [2]: [sum(sales#40)#234 AS sum_sales#84, sum(number_sales#41)#235 AS number_sales#85]
Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240]
Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8]
Functions [2]: [sum(sales#39), sum(number_sales#40)]
Aggregate Attributes [2]: [sum(sales#39)#242, sum(number_sales#40)#243]
Results [2]: [sum(sales#39)#242 AS sum_sales#84, sum(number_sales#40)#243 AS number_sales#85]
(199) HashAggregate [codegen id : 409]
Input [2]: [sum_sales#84, number_sales#85]
Keys: []
Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)]
Aggregate Attributes [3]: [sum#236, isEmpty#237, sum#238]
Results [3]: [sum#239, isEmpty#240, sum#241]
Aggregate Attributes [3]: [sum#244, isEmpty#245, sum#246]
Results [3]: [sum#247, isEmpty#248, sum#249]
(200) Exchange
Input [3]: [sum#239, isEmpty#240, sum#241]
Arguments: SinglePartition, true, [id=#242]
Input [3]: [sum#247, isEmpty#248, sum#249]
Arguments: SinglePartition, true, [id=#250]
(201) HashAggregate [codegen id : 410]
Input [3]: [sum#239, isEmpty#240, sum#241]
Input [3]: [sum#247, isEmpty#248, sum#249]
Keys: []
Functions [2]: [sum(sum_sales#84), sum(number_sales#85)]
Aggregate Attributes [2]: [sum(sum_sales#84)#243, sum(number_sales#85)#244]
Results [6]: [null AS channel#245, null AS i_brand_id#246, null AS i_class_id#247, null AS i_category_id#248, sum(sum_sales#84)#243 AS sum(sum_sales)#249, sum(number_sales#85)#244 AS sum(number_sales)#250]
Aggregate Attributes [2]: [sum(sum_sales#84)#251, sum(number_sales#85)#252]
Results [6]: [null AS channel#253, null AS i_brand_id#254, null AS i_class_id#255, null AS i_category_id#256, sum(sum_sales#84)#251 AS sum(sum_sales)#257, sum(number_sales#85)#252 AS sum(number_sales)#258]
(202) Union
(203) HashAggregate [codegen id : 411]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(204) Exchange
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#251]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#259]
(205) HashAggregate [codegen id : 412]
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
(206) TakeOrderedAndProject
Input [6]: [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: 100, [channel#39 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#39, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85]
===== Subqueries =====
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#43, [id=#44]
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43]
* HashAggregate (236)
+- Exchange (235)
+- * HashAggregate (234)
@ -1247,7 +1247,7 @@ Input [2]: [d_date_sk#10, d_year#11]
(214) BroadcastExchange
Input [1]: [d_date_sk#10]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#252]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260]
(215) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#1]
@ -1255,7 +1255,7 @@ Right keys [1]: [d_date_sk#10]
Join condition: None
(216) Project [codegen id : 2]
Output [2]: [ss_quantity#3 AS quantity#253, ss_list_price#4 AS list_price#254]
Output [2]: [ss_quantity#3 AS quantity#261, ss_list_price#4 AS list_price#262]
Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10]
(217) Scan parquet default.catalog_sales
@ -1292,7 +1292,7 @@ Input [2]: [d_date_sk#10, d_year#11]
(224) BroadcastExchange
Input [1]: [d_date_sk#10]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#255]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#263]
(225) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#16]
@ -1300,7 +1300,7 @@ Right keys [1]: [d_date_sk#10]
Join condition: None
(226) Project [codegen id : 4]
Output [2]: [cs_quantity#45 AS quantity#256, cs_list_price#46 AS list_price#257]
Output [2]: [cs_quantity#45 AS quantity#264, cs_list_price#46 AS list_price#265]
Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10]
(227) Scan parquet default.web_sales
@ -1326,55 +1326,55 @@ Right keys [1]: [d_date_sk#10]
Join condition: None
(232) Project [codegen id : 6]
Output [2]: [ws_quantity#60 AS quantity#258, ws_list_price#61 AS list_price#259]
Output [2]: [ws_quantity#60 AS quantity#266, ws_list_price#61 AS list_price#267]
Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10]
(233) Union
(234) HashAggregate [codegen id : 7]
Input [2]: [quantity#253, list_price#254]
Input [2]: [quantity#261, list_price#262]
Keys: []
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#260, count#261]
Results [2]: [sum#262, count#263]
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#268, count#269]
Results [2]: [sum#270, count#271]
(235) Exchange
Input [2]: [sum#262, count#263]
Arguments: SinglePartition, true, [id=#264]
Input [2]: [sum#270, count#271]
Arguments: SinglePartition, true, [id=#272]
(236) HashAggregate [codegen id : 8]
Input [2]: [sum#262, count#263]
Input [2]: [sum#270, count#271]
Keys: []
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))#265]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#253 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#254 as decimal(12,2)))), DecimalType(18,2), true))#265 AS average_sales#266]
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273 AS average_sales#274]
Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:7 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:7 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:8 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:8 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:9 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:9 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:10 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:10 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:11 Hosting operator id = 166 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:11 Hosting operator id = 166 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:12 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:12 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:13 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:13 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:14 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:14 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]
Subquery:15 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44]
Subquery:15 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43]

View file

@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (26)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
Subquery #1
WholeStageCodegen (8)
@ -90,7 +90,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price]
InputAdapter
ReusedExchange [d_date_sk] #19
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #6
WholeStageCodegen (25)
@ -202,10 +202,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (52)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #20
WholeStageCodegen (51)
@ -226,10 +226,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
ReusedExchange [d_date_sk] #16
WholeStageCodegen (78)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
Exchange [i_brand_id,i_class_id,i_category_id] #21
WholeStageCodegen (77)
@ -263,24 +263,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (106)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (132)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20
WholeStageCodegen (158)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #21
WholeStageCodegen (244)
@ -297,24 +297,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (189)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (215)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20
WholeStageCodegen (241)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #21
WholeStageCodegen (327)
@ -331,24 +331,24 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (272)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (298)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20
WholeStageCodegen (324)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #21
WholeStageCodegen (410)
@ -365,23 +365,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
InputAdapter
Union
WholeStageCodegen (355)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #6
WholeStageCodegen (381)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20
WholeStageCodegen (407)
Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #1
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,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]
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] #21