[SPARK-35742][SQL] Expression.semanticEquals should be symmetrical

### What changes were proposed in this pull request?

Currently, there are some expressions that overwrite `semanticEquals`, which makes it not symmetrical. Ideally, expressions should overwrite `canonicalized` instead of `semanticEquals`.

This PR marks `semanticEquals` as final, and implement `canonicalized` for the few expressions that overwrote `semanticEquals` before.

### Why are the changes needed?

To avoid subtle bugs (I haven't found a real bug yet).

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

no

### How was this patch tested?

a new test

Closes #32885 from cloud-fan/attr.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
Wenchen Fan 2021-06-15 08:53:04 +00:00
parent b9aeeb4e6c
commit a50bd8f810
21 changed files with 1060 additions and 2356 deletions

View file

@ -238,7 +238,7 @@ abstract class Expression extends TreeNode[Expression] {
*
* See [[Canonicalize]] for more details.
*/
def semanticEquals(other: Expression): Boolean =
final def semanticEquals(other: Expression): Boolean =
deterministic && other.deterministic && canonicalized == other.canonicalized
/**

View file

@ -278,11 +278,6 @@ case class AttributeReference(
case _ => false
}
override def semanticEquals(other: Expression): Boolean = other match {
case ar: AttributeReference => sameRef(ar)
case _ => false
}
override def semanticHash(): Int = {
this.exprId.hashCode()
}

View file

@ -76,13 +76,6 @@ abstract class SubqueryExpression(
AttributeSet.fromAttributeSets(outerAttrs.map(_.references))
override def children: Seq[Expression] = outerAttrs ++ joinCond
override def withNewPlan(plan: LogicalPlan): SubqueryExpression
override def semanticEquals(o: Expression): Boolean = o match {
case p: SubqueryExpression =>
this.getClass.getName.equals(p.getClass.getName) && plan.sameResult(p.plan) &&
children.length == p.children.length &&
children.zip(p.children).forall(p => p._1.semanticEquals(p._2))
case _ => false
}
}
object SubqueryExpression {

View file

@ -170,4 +170,11 @@ class CanonicalizeSuite extends SparkFunSuite {
assert(nestedExpr2.canonicalized != nestedExpr3.canonicalized)
}
}
test("SPARK-35742: Expression.semanticEquals should be symmetrical") {
val attr = AttributeReference("col", IntegerType)()
val expr = PromotePrecision(attr)
assert(expr.semanticEquals(attr))
assert(attr.semanticEquals(expr))
}
}

View file

@ -20,6 +20,7 @@ package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.QueryPlan
/**
* Similar to [[SubqueryBroadcastExec]], this node is used to store the
@ -40,6 +41,11 @@ case class SubqueryAdaptiveBroadcastExec(
"SubqueryAdaptiveBroadcastExec does not support the execute() code path.")
}
protected override def doCanonicalize(): SparkPlan = {
val keys = buildKeys.map(k => QueryPlan.normalizeExpressions(k, child.output))
copy(name = "dpp", buildKeys = keys, child = child.canonicalized)
}
override protected def withNewChildInternal(newChild: SparkPlan): SubqueryAdaptiveBroadcastExec =
copy(child = newChild)
}

View file

@ -73,7 +73,9 @@ case class HashAggregateExec(
// This is for testing. We force TungstenAggregationIterator to fall back to the unsafe row hash
// map and/or the sort-based aggregation once it has processed a given number of input rows.
private val testFallbackStartsAt: Option[(Int, Int)] = {
sqlContext.getConf("spark.sql.TungstenAggregate.testFallbackStartsAt", null) match {
Option(sqlContext).map { sc =>
sc.getConf("spark.sql.TungstenAggregate.testFallbackStartsAt", null)
}.orNull match {
case null | "" => None
case fallbackStartsAt =>
val splits = fallbackStartsAt.split(",").map(_.trim)

View file

@ -71,9 +71,8 @@ case class ScalarSubquery(
override def toString: String = plan.simpleString(SQLConf.get.maxToStringFields)
override def withNewPlan(query: BaseSubqueryExec): ScalarSubquery = copy(plan = query)
override def semanticEquals(other: Expression): Boolean = other match {
case s: ScalarSubquery => plan.sameResult(s.plan)
case _ => false
override lazy val canonicalized: Expression = {
ScalarSubquery(plan.canonicalized.asInstanceOf[BaseSubqueryExec], ExprId(0))
}
// the first column in first row from `query`.
@ -127,11 +126,6 @@ case class InSubqueryExec(
override def withNewPlan(plan: BaseSubqueryExec): InSubqueryExec = copy(plan = plan)
final override def nodePatternsInternal: Seq[TreePattern] = Seq(IN_SUBQUERY_EXEC)
override def semanticEquals(other: Expression): Boolean = other match {
case in: InSubqueryExec => child.semanticEquals(in.child) && plan.sameResult(in.plan)
case _ => false
}
def updateResult(): Unit = {
val rows = plan.executeCollect()
result = if (plan.output.length > 1) {

View file

@ -1,6 +1,6 @@
== Physical Plan ==
TakeOrderedAndProject (103)
+- Union (102)
TakeOrderedAndProject (95)
+- Union (94)
:- * HashAggregate (70)
: +- Exchange (69)
: +- * HashAggregate (68)
@ -71,13 +71,13 @@ TakeOrderedAndProject (103)
: +- * Filter (63)
: +- * ColumnarToRow (62)
: +- Scan parquet default.date_dim (61)
+- * HashAggregate (101)
+- Exchange (100)
+- * HashAggregate (99)
+- * Project (98)
+- * BroadcastHashJoin Inner BuildRight (97)
:- * Project (95)
: +- * BroadcastHashJoin Inner BuildRight (94)
+- * HashAggregate (93)
+- Exchange (92)
+- * HashAggregate (91)
+- * Project (90)
+- * BroadcastHashJoin Inner BuildRight (89)
:- * Project (87)
: +- * BroadcastHashJoin Inner BuildRight (86)
: :- * SortMergeJoin LeftSemi (84)
: : :- * Sort (78)
: : : +- Exchange (77)
@ -92,16 +92,8 @@ TakeOrderedAndProject (103)
: : +- * Filter (81)
: : +- * HashAggregate (80)
: : +- ReusedExchange (79)
: +- BroadcastExchange (93)
: +- * SortMergeJoin LeftSemi (92)
: :- * Sort (86)
: : +- ReusedExchange (85)
: +- * Sort (91)
: +- * Project (90)
: +- * Filter (89)
: +- * HashAggregate (88)
: +- ReusedExchange (87)
+- ReusedExchange (96)
: +- ReusedExchange (85)
+- ReusedExchange (88)
(1) Scan parquet default.catalog_sales
@ -492,245 +484,210 @@ Left keys [1]: [ws_bill_customer_sk#60]
Right keys [1]: [c_customer_sk#65]
Join condition: None
(85) ReusedExchange [Reuses operator id: 50]
(85) ReusedExchange [Reuses operator id: unknown]
Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74]
(86) Sort [codegen id : 29]
Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74]
Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0
(87) ReusedExchange [Reuses operator id: 41]
Output [3]: [c_customer_sk#65, sum#66, isEmpty#67]
(88) HashAggregate [codegen id : 32]
Input [3]: [c_customer_sk#65, sum#66, isEmpty#67]
Keys [1]: [c_customer_sk#65]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70]
Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71]
(89) Filter [codegen id : 32]
Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true)))
(90) Project [codegen id : 32]
Output [1]: [c_customer_sk#65]
Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71]
(91) Sort [codegen id : 32]
Input [1]: [c_customer_sk#65]
Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0
(92) SortMergeJoin [codegen id : 33]
Left keys [1]: [c_customer_sk#72]
Right keys [1]: [c_customer_sk#65]
Join condition: None
(93) BroadcastExchange
Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75]
(94) BroadcastHashJoin [codegen id : 35]
(86) BroadcastHashJoin [codegen id : 35]
Left keys [1]: [ws_bill_customer_sk#60]
Right keys [1]: [c_customer_sk#72]
Join condition: None
(95) Project [codegen id : 35]
(87) Project [codegen id : 35]
Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74]
Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74]
(96) ReusedExchange [Reuses operator id: 65]
Output [1]: [d_date_sk#76]
(88) ReusedExchange [Reuses operator id: 65]
Output [1]: [d_date_sk#75]
(97) BroadcastHashJoin [codegen id : 35]
(89) BroadcastHashJoin [codegen id : 35]
Left keys [1]: [ws_sold_date_sk#63]
Right keys [1]: [d_date_sk#76]
Right keys [1]: [d_date_sk#75]
Join condition: None
(98) Project [codegen id : 35]
(90) Project [codegen id : 35]
Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74]
Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76]
Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#75]
(99) HashAggregate [codegen id : 35]
(91) HashAggregate [codegen id : 35]
Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74]
Keys [2]: [c_last_name#74, c_first_name#73]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#77, isEmpty#78]
Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80]
Aggregate Attributes [2]: [sum#76, isEmpty#77]
Results [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79]
(100) Exchange
Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80]
Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81]
(92) Exchange
Input [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79]
Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#80]
(101) HashAggregate [codegen id : 36]
Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80]
(93) HashAggregate [codegen id : 36]
Input [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79]
Keys [2]: [c_last_name#74, c_first_name#73]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82]
Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#81]
Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#82]
(102) Union
(94) Union
(103) TakeOrderedAndProject
(95) TakeOrderedAndProject
Input [3]: [c_last_name#45, c_first_name#44, sales#58]
Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6
ReusedExchange (104)
ReusedExchange (96)
(104) ReusedExchange [Reuses operator id: 65]
(96) ReusedExchange [Reuses operator id: 65]
Output [1]: [d_date_sk#48]
Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9
ReusedExchange (105)
ReusedExchange (97)
(105) ReusedExchange [Reuses operator id: 11]
(97) ReusedExchange [Reuses operator id: 11]
Output [2]: [d_date_sk#10, d_date#11]
Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42]
* HashAggregate (127)
+- Exchange (126)
+- * HashAggregate (125)
+- * HashAggregate (124)
+- Exchange (123)
+- * HashAggregate (122)
+- * Project (121)
+- * BroadcastHashJoin Inner BuildRight (120)
:- * Project (114)
: +- * BroadcastHashJoin Inner BuildRight (113)
: :- * Filter (108)
: : +- * ColumnarToRow (107)
: : +- Scan parquet default.store_sales (106)
: +- BroadcastExchange (112)
: +- * Filter (111)
: +- * ColumnarToRow (110)
: +- Scan parquet default.customer (109)
+- BroadcastExchange (119)
+- * Project (118)
+- * Filter (117)
+- * ColumnarToRow (116)
+- Scan parquet default.date_dim (115)
* HashAggregate (119)
+- Exchange (118)
+- * HashAggregate (117)
+- * HashAggregate (116)
+- Exchange (115)
+- * HashAggregate (114)
+- * Project (113)
+- * BroadcastHashJoin Inner BuildRight (112)
:- * Project (106)
: +- * BroadcastHashJoin Inner BuildRight (105)
: :- * Filter (100)
: : +- * ColumnarToRow (99)
: : +- Scan parquet default.store_sales (98)
: +- BroadcastExchange (104)
: +- * Filter (103)
: +- * ColumnarToRow (102)
: +- Scan parquet default.customer (101)
+- BroadcastExchange (111)
+- * Project (110)
+- * Filter (109)
+- * ColumnarToRow (108)
+- Scan parquet default.date_dim (107)
(106) Scan parquet default.store_sales
Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87]
(98) Scan parquet default.store_sales
Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)]
PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)]
PushedFilters: [IsNotNull(ss_customer_sk)]
ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)>
(107) ColumnarToRow [codegen id : 3]
Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87]
(99) ColumnarToRow [codegen id : 3]
Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86]
(108) Filter [codegen id : 3]
Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87]
Condition : isnotnull(ss_customer_sk#84)
(100) Filter [codegen id : 3]
Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86]
Condition : isnotnull(ss_customer_sk#83)
(109) Scan parquet default.customer
Output [1]: [c_customer_sk#89]
(101) Scan parquet default.customer
Output [1]: [c_customer_sk#88]
Batched: true
Location [not included in comparison]/{warehouse_dir}/customer]
PushedFilters: [IsNotNull(c_customer_sk)]
ReadSchema: struct<c_customer_sk:int>
(110) ColumnarToRow [codegen id : 1]
Input [1]: [c_customer_sk#89]
(102) ColumnarToRow [codegen id : 1]
Input [1]: [c_customer_sk#88]
(111) Filter [codegen id : 1]
Input [1]: [c_customer_sk#89]
Condition : isnotnull(c_customer_sk#89)
(103) Filter [codegen id : 1]
Input [1]: [c_customer_sk#88]
Condition : isnotnull(c_customer_sk#88)
(112) BroadcastExchange
Input [1]: [c_customer_sk#89]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90]
(104) BroadcastExchange
Input [1]: [c_customer_sk#88]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#89]
(113) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [ss_customer_sk#84]
Right keys [1]: [c_customer_sk#89]
(105) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [ss_customer_sk#83]
Right keys [1]: [c_customer_sk#88]
Join condition: None
(114) Project [codegen id : 3]
Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89]
Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89]
(106) Project [codegen id : 3]
Output [4]: [ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88]
Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88]
(115) Scan parquet default.date_dim
Output [2]: [d_date_sk#91, d_year#92]
(107) Scan parquet default.date_dim
Output [2]: [d_date_sk#90, d_year#91]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
(116) ColumnarToRow [codegen id : 2]
Input [2]: [d_date_sk#91, d_year#92]
(108) ColumnarToRow [codegen id : 2]
Input [2]: [d_date_sk#90, d_year#91]
(117) Filter [codegen id : 2]
Input [2]: [d_date_sk#91, d_year#92]
Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91))
(109) Filter [codegen id : 2]
Input [2]: [d_date_sk#90, d_year#91]
Condition : (d_year#91 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#90))
(118) Project [codegen id : 2]
Output [1]: [d_date_sk#91]
Input [2]: [d_date_sk#91, d_year#92]
(110) Project [codegen id : 2]
Output [1]: [d_date_sk#90]
Input [2]: [d_date_sk#90, d_year#91]
(119) BroadcastExchange
Input [1]: [d_date_sk#91]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93]
(111) BroadcastExchange
Input [1]: [d_date_sk#90]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92]
(120) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [ss_sold_date_sk#87]
Right keys [1]: [d_date_sk#91]
(112) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [ss_sold_date_sk#86]
Right keys [1]: [d_date_sk#90]
Join condition: None
(121) Project [codegen id : 3]
Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89]
Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91]
(113) Project [codegen id : 3]
Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#88]
Input [5]: [ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88, d_date_sk#90]
(122) HashAggregate [codegen id : 3]
Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89]
Keys [1]: [c_customer_sk#89]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#94, isEmpty#95]
Results [3]: [c_customer_sk#89, sum#96, isEmpty#97]
(114) HashAggregate [codegen id : 3]
Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#88]
Keys [1]: [c_customer_sk#88]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#93, isEmpty#94]
Results [3]: [c_customer_sk#88, sum#95, isEmpty#96]
(123) Exchange
Input [3]: [c_customer_sk#89, sum#96, isEmpty#97]
Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98]
(115) Exchange
Input [3]: [c_customer_sk#88, sum#95, isEmpty#96]
Arguments: hashpartitioning(c_customer_sk#88, 5), ENSURE_REQUIREMENTS, [id=#97]
(124) HashAggregate [codegen id : 4]
Input [3]: [c_customer_sk#89, sum#96, isEmpty#97]
Keys [1]: [c_customer_sk#89]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99]
Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100]
(116) HashAggregate [codegen id : 4]
Input [3]: [c_customer_sk#88, sum#95, isEmpty#96]
Keys [1]: [c_customer_sk#88]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98]
Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99]
(125) HashAggregate [codegen id : 4]
Input [1]: [csales#100]
(117) HashAggregate [codegen id : 4]
Input [1]: [csales#99]
Keys: []
Functions [1]: [partial_max(csales#100)]
Aggregate Attributes [1]: [max#101]
Results [1]: [max#102]
Functions [1]: [partial_max(csales#99)]
Aggregate Attributes [1]: [max#100]
Results [1]: [max#101]
(126) Exchange
Input [1]: [max#102]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103]
(118) Exchange
Input [1]: [max#101]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102]
(127) HashAggregate [codegen id : 5]
Input [1]: [max#102]
(119) HashAggregate [codegen id : 5]
Input [1]: [max#101]
Keys: []
Functions [1]: [max(csales#100)]
Aggregate Attributes [1]: [max(csales#100)#104]
Results [1]: [max(csales#100)#104 AS tpcds_cmax#105]
Functions [1]: [max(csales#99)]
Aggregate Attributes [1]: [max(csales#99)#103]
Results [1]: [max(csales#99)#103 AS tpcds_cmax#104]
Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88
ReusedExchange (128)
Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87
ReusedExchange (120)
(128) ReusedExchange [Reuses operator id: 119]
Output [1]: [d_date_sk#91]
(120) ReusedExchange [Reuses operator id: 111]
Output [1]: [d_date_sk#90]
Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42]
@ -738,6 +695,4 @@ Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN d
Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42]
Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42]

View file

@ -188,22 +188,6 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
InputAdapter
ReusedExchange [c_customer_sk,sum,isEmpty] #8
InputAdapter
BroadcastExchange #19
WholeStageCodegen (33)
SortMergeJoin [c_customer_sk,c_customer_sk]
InputAdapter
WholeStageCodegen (29)
Sort [c_customer_sk]
InputAdapter
ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15
InputAdapter
WholeStageCodegen (32)
Sort [c_customer_sk]
Project [c_customer_sk]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [tpcds_cmax] #3
HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty]
InputAdapter
ReusedExchange [c_customer_sk,sum,isEmpty] #8
ReusedExchange [c_customer_sk,c_first_name,c_last_name] #19
InputAdapter
ReusedExchange [d_date_sk] #3

View file

@ -1,11 +1,11 @@
== Physical Plan ==
TakeOrderedAndProject (36)
+- * Project (35)
+- * BroadcastHashJoin Inner BuildRight (34)
:- * Project (32)
: +- * BroadcastHashJoin Inner BuildRight (31)
: :- * Project (26)
: : +- * BroadcastHashJoin Inner BuildRight (25)
TakeOrderedAndProject (32)
+- * Project (31)
+- * BroadcastHashJoin Inner BuildRight (30)
:- * Project (28)
: +- * BroadcastHashJoin Inner BuildRight (27)
: :- * Project (22)
: : +- * BroadcastHashJoin Inner BuildRight (21)
: : :- * Project (14)
: : : +- * Filter (13)
: : : +- Window (12)
@ -20,21 +20,17 @@ TakeOrderedAndProject (36)
: : : +- * Filter (3)
: : : +- * ColumnarToRow (2)
: : : +- Scan parquet default.store_sales (1)
: : +- BroadcastExchange (24)
: : +- * Project (23)
: : +- * Filter (22)
: : +- Window (21)
: : +- * Sort (20)
: : +- Exchange (19)
: : +- * Project (18)
: : +- * Filter (17)
: : +- * HashAggregate (16)
: : +- ReusedExchange (15)
: +- BroadcastExchange (30)
: +- * Filter (29)
: +- * ColumnarToRow (28)
: +- Scan parquet default.item (27)
+- ReusedExchange (33)
: : +- BroadcastExchange (20)
: : +- * Project (19)
: : +- * Filter (18)
: : +- Window (17)
: : +- * Sort (16)
: : +- ReusedExchange (15)
: +- BroadcastExchange (26)
: +- * Filter (25)
: +- * ColumnarToRow (24)
: +- Scan parquet default.item (23)
+- ReusedExchange (29)
(1) Scan parquet default.store_sales
@ -101,148 +97,127 @@ Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11))
Output [2]: [item_sk#11, rnk#17]
Input [3]: [item_sk#11, rank_col#12, rnk#17]
(15) ReusedExchange [Reuses operator id: 6]
Output [3]: [ss_item_sk#18, sum#19, count#20]
(15) ReusedExchange [Reuses operator id: 10]
Output [2]: [item_sk#18, rank_col#19]
(16) HashAggregate [codegen id : 5]
Input [3]: [ss_item_sk#18, sum#19, count#20]
Keys [1]: [ss_item_sk#18]
Functions [1]: [avg(UnscaledValue(ss_net_profit#21))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22]
Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25]
(16) Sort [codegen id : 6]
Input [2]: [item_sk#18, rank_col#19]
Arguments: [rank_col#19 DESC NULLS LAST], false, 0
(17) Filter [codegen id : 5]
Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25]
Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true)))
(17) Window
Input [2]: [item_sk#18, rank_col#19]
Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST]
(18) Project [codegen id : 5]
Output [2]: [item_sk#23, rank_col#24]
Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25]
(18) Filter [codegen id : 7]
Input [3]: [item_sk#18, rank_col#19, rnk#20]
Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18))
(19) Exchange
Input [2]: [item_sk#23, rank_col#24]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26]
(19) Project [codegen id : 7]
Output [2]: [item_sk#18, rnk#20]
Input [3]: [item_sk#18, rank_col#19, rnk#20]
(20) Sort [codegen id : 6]
Input [2]: [item_sk#23, rank_col#24]
Arguments: [rank_col#24 DESC NULLS LAST], false, 0
(20) BroadcastExchange
Input [2]: [item_sk#18, rnk#20]
Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#21]
(21) Window
Input [2]: [item_sk#23, rank_col#24]
Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST]
(22) Filter [codegen id : 7]
Input [3]: [item_sk#23, rank_col#24, rnk#27]
Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23))
(23) Project [codegen id : 7]
Output [2]: [item_sk#23, rnk#27]
Input [3]: [item_sk#23, rank_col#24, rnk#27]
(24) BroadcastExchange
Input [2]: [item_sk#23, rnk#27]
Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28]
(25) BroadcastHashJoin [codegen id : 10]
(21) BroadcastHashJoin [codegen id : 10]
Left keys [1]: [rnk#17]
Right keys [1]: [rnk#27]
Right keys [1]: [rnk#20]
Join condition: None
(26) Project [codegen id : 10]
Output [3]: [item_sk#11, rnk#17, item_sk#23]
Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27]
(22) Project [codegen id : 10]
Output [3]: [item_sk#11, rnk#17, item_sk#18]
Input [4]: [item_sk#11, rnk#17, item_sk#18, rnk#20]
(27) Scan parquet default.item
Output [2]: [i_item_sk#29, i_product_name#30]
(23) Scan parquet default.item
Output [2]: [i_item_sk#22, i_product_name#23]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_product_name:string>
(28) ColumnarToRow [codegen id : 8]
Input [2]: [i_item_sk#29, i_product_name#30]
(24) ColumnarToRow [codegen id : 8]
Input [2]: [i_item_sk#22, i_product_name#23]
(29) Filter [codegen id : 8]
Input [2]: [i_item_sk#29, i_product_name#30]
Condition : isnotnull(i_item_sk#29)
(25) Filter [codegen id : 8]
Input [2]: [i_item_sk#22, i_product_name#23]
Condition : isnotnull(i_item_sk#22)
(30) BroadcastExchange
Input [2]: [i_item_sk#29, i_product_name#30]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31]
(26) BroadcastExchange
Input [2]: [i_item_sk#22, i_product_name#23]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24]
(31) BroadcastHashJoin [codegen id : 10]
(27) BroadcastHashJoin [codegen id : 10]
Left keys [1]: [item_sk#11]
Right keys [1]: [i_item_sk#29]
Right keys [1]: [i_item_sk#22]
Join condition: None
(32) Project [codegen id : 10]
Output [3]: [rnk#17, item_sk#23, i_product_name#30]
Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#29, i_product_name#30]
(28) Project [codegen id : 10]
Output [3]: [rnk#17, item_sk#18, i_product_name#23]
Input [5]: [item_sk#11, rnk#17, item_sk#18, i_item_sk#22, i_product_name#23]
(33) ReusedExchange [Reuses operator id: 30]
Output [2]: [i_item_sk#32, i_product_name#33]
(29) ReusedExchange [Reuses operator id: 26]
Output [2]: [i_item_sk#25, i_product_name#26]
(34) BroadcastHashJoin [codegen id : 10]
Left keys [1]: [item_sk#23]
Right keys [1]: [i_item_sk#32]
(30) BroadcastHashJoin [codegen id : 10]
Left keys [1]: [item_sk#18]
Right keys [1]: [i_item_sk#25]
Join condition: None
(35) Project [codegen id : 10]
Output [3]: [rnk#17, i_product_name#30 AS best_performing#34, i_product_name#33 AS worst_performing#35]
Input [5]: [rnk#17, item_sk#23, i_product_name#30, i_item_sk#32, i_product_name#33]
(31) Project [codegen id : 10]
Output [3]: [rnk#17, i_product_name#23 AS best_performing#27, i_product_name#26 AS worst_performing#28]
Input [5]: [rnk#17, item_sk#18, i_product_name#23, i_item_sk#25, i_product_name#26]
(36) TakeOrderedAndProject
Input [3]: [rnk#17, best_performing#34, worst_performing#35]
Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#34, worst_performing#35]
(32) TakeOrderedAndProject
Input [3]: [rnk#17, best_performing#27, worst_performing#28]
Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#27, worst_performing#28]
===== Subqueries =====
Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#14, [id=#15]
* HashAggregate (43)
+- Exchange (42)
+- * HashAggregate (41)
+- * Project (40)
+- * Filter (39)
+- * ColumnarToRow (38)
+- Scan parquet default.store_sales (37)
* HashAggregate (39)
+- Exchange (38)
+- * HashAggregate (37)
+- * Project (36)
+- * Filter (35)
+- * ColumnarToRow (34)
+- Scan parquet default.store_sales (33)
(37) Scan parquet default.store_sales
Output [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39]
(33) Scan parquet default.store_sales
Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
Batched: true
Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)]
ReadSchema: struct<ss_addr_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)>
(38) ColumnarToRow [codegen id : 1]
Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39]
(34) ColumnarToRow [codegen id : 1]
Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
(39) Filter [codegen id : 1]
Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39]
Condition : ((isnotnull(ss_store_sk#37) AND (ss_store_sk#37 = 4)) AND isnull(ss_addr_sk#36))
(35) Filter [codegen id : 1]
Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29))
(40) Project [codegen id : 1]
Output [2]: [ss_store_sk#37, ss_net_profit#38]
Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39]
(36) Project [codegen id : 1]
Output [2]: [ss_store_sk#30, ss_net_profit#31]
Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
(41) HashAggregate [codegen id : 1]
Input [2]: [ss_store_sk#37, ss_net_profit#38]
Keys [1]: [ss_store_sk#37]
Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#38))]
Aggregate Attributes [2]: [sum#40, count#41]
Results [3]: [ss_store_sk#37, sum#42, count#43]
(37) HashAggregate [codegen id : 1]
Input [2]: [ss_store_sk#30, ss_net_profit#31]
Keys [1]: [ss_store_sk#30]
Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))]
Aggregate Attributes [2]: [sum#33, count#34]
Results [3]: [ss_store_sk#30, sum#35, count#36]
(42) Exchange
Input [3]: [ss_store_sk#37, sum#42, count#43]
Arguments: hashpartitioning(ss_store_sk#37, 5), ENSURE_REQUIREMENTS, [id=#44]
(38) Exchange
Input [3]: [ss_store_sk#30, sum#35, count#36]
Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [id=#37]
(43) HashAggregate [codegen id : 2]
Input [3]: [ss_store_sk#37, sum#42, count#43]
Keys [1]: [ss_store_sk#37]
Functions [1]: [avg(UnscaledValue(ss_net_profit#38))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#38))#45]
Results [1]: [cast((avg(UnscaledValue(ss_net_profit#38))#45 / 100.0) as decimal(11,6)) AS rank_col#46]
Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15]
(39) HashAggregate [codegen id : 2]
Input [3]: [ss_store_sk#30, sum#35, count#36]
Keys [1]: [ss_store_sk#30]
Functions [1]: [avg(UnscaledValue(ss_net_profit#31))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#38]
Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#38 / 100.0) as decimal(11,6)) AS rank_col#39]

View file

@ -49,20 +49,13 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (6)
Sort [rank_col]
InputAdapter
Exchange #5
WholeStageCodegen (5)
Project [item_sk,rank_col]
Filter [avg(ss_net_profit)]
ReusedSubquery [rank_col] #1
HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count]
InputAdapter
ReusedExchange [ss_item_sk,sum,count] #2
ReusedExchange [item_sk,rank_col] #1
InputAdapter
BroadcastExchange #6
BroadcastExchange #5
WholeStageCodegen (8)
Filter [i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_product_name]
InputAdapter
ReusedExchange [i_item_sk,i_product_name] #6
ReusedExchange [i_item_sk,i_product_name] #5

View file

@ -1,11 +1,11 @@
== Physical Plan ==
TakeOrderedAndProject (37)
+- * Project (36)
+- * BroadcastHashJoin Inner BuildRight (35)
:- * Project (33)
: +- * BroadcastHashJoin Inner BuildRight (32)
: :- * Project (27)
: : +- * SortMergeJoin Inner (26)
TakeOrderedAndProject (33)
+- * Project (32)
+- * BroadcastHashJoin Inner BuildRight (31)
:- * Project (29)
: +- * BroadcastHashJoin Inner BuildRight (28)
: :- * Project (23)
: : +- * SortMergeJoin Inner (22)
: : :- * Sort (15)
: : : +- * Project (14)
: : : +- * Filter (13)
@ -21,21 +21,17 @@ TakeOrderedAndProject (37)
: : : +- * Filter (3)
: : : +- * ColumnarToRow (2)
: : : +- Scan parquet default.store_sales (1)
: : +- * Sort (25)
: : +- * Project (24)
: : +- * Filter (23)
: : +- Window (22)
: : +- * Sort (21)
: : +- Exchange (20)
: : +- * Project (19)
: : +- * Filter (18)
: : +- * HashAggregate (17)
: : +- ReusedExchange (16)
: +- BroadcastExchange (31)
: +- * Filter (30)
: +- * ColumnarToRow (29)
: +- Scan parquet default.item (28)
+- ReusedExchange (34)
: : +- * Sort (21)
: : +- * Project (20)
: : +- * Filter (19)
: : +- Window (18)
: : +- * Sort (17)
: : +- ReusedExchange (16)
: +- BroadcastExchange (27)
: +- * Filter (26)
: +- * ColumnarToRow (25)
: +- Scan parquet default.item (24)
+- ReusedExchange (30)
(1) Scan parquet default.store_sales
@ -106,148 +102,127 @@ Input [3]: [item_sk#11, rank_col#12, rnk#17]
Input [2]: [item_sk#11, rnk#17]
Arguments: [rnk#17 ASC NULLS FIRST], false, 0
(16) ReusedExchange [Reuses operator id: 6]
Output [3]: [ss_item_sk#18, sum#19, count#20]
(16) ReusedExchange [Reuses operator id: 10]
Output [2]: [item_sk#18, rank_col#19]
(17) HashAggregate [codegen id : 6]
Input [3]: [ss_item_sk#18, sum#19, count#20]
Keys [1]: [ss_item_sk#18]
Functions [1]: [avg(UnscaledValue(ss_net_profit#21))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22]
Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25]
(17) Sort [codegen id : 7]
Input [2]: [item_sk#18, rank_col#19]
Arguments: [rank_col#19 DESC NULLS LAST], false, 0
(18) Filter [codegen id : 6]
Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25]
Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true)))
(18) Window
Input [2]: [item_sk#18, rank_col#19]
Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST]
(19) Project [codegen id : 6]
Output [2]: [item_sk#23, rank_col#24]
Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25]
(19) Filter [codegen id : 8]
Input [3]: [item_sk#18, rank_col#19, rnk#20]
Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18))
(20) Exchange
Input [2]: [item_sk#23, rank_col#24]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26]
(20) Project [codegen id : 8]
Output [2]: [item_sk#18, rnk#20]
Input [3]: [item_sk#18, rank_col#19, rnk#20]
(21) Sort [codegen id : 7]
Input [2]: [item_sk#23, rank_col#24]
Arguments: [rank_col#24 DESC NULLS LAST], false, 0
(21) Sort [codegen id : 8]
Input [2]: [item_sk#18, rnk#20]
Arguments: [rnk#20 ASC NULLS FIRST], false, 0
(22) Window
Input [2]: [item_sk#23, rank_col#24]
Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST]
(23) Filter [codegen id : 8]
Input [3]: [item_sk#23, rank_col#24, rnk#27]
Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23))
(24) Project [codegen id : 8]
Output [2]: [item_sk#23, rnk#27]
Input [3]: [item_sk#23, rank_col#24, rnk#27]
(25) Sort [codegen id : 8]
Input [2]: [item_sk#23, rnk#27]
Arguments: [rnk#27 ASC NULLS FIRST], false, 0
(26) SortMergeJoin [codegen id : 11]
(22) SortMergeJoin [codegen id : 11]
Left keys [1]: [rnk#17]
Right keys [1]: [rnk#27]
Right keys [1]: [rnk#20]
Join condition: None
(27) Project [codegen id : 11]
Output [3]: [item_sk#11, rnk#17, item_sk#23]
Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27]
(23) Project [codegen id : 11]
Output [3]: [item_sk#11, rnk#17, item_sk#18]
Input [4]: [item_sk#11, rnk#17, item_sk#18, rnk#20]
(28) Scan parquet default.item
Output [2]: [i_item_sk#28, i_product_name#29]
(24) Scan parquet default.item
Output [2]: [i_item_sk#21, i_product_name#22]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_product_name:string>
(29) ColumnarToRow [codegen id : 9]
Input [2]: [i_item_sk#28, i_product_name#29]
(25) ColumnarToRow [codegen id : 9]
Input [2]: [i_item_sk#21, i_product_name#22]
(30) Filter [codegen id : 9]
Input [2]: [i_item_sk#28, i_product_name#29]
Condition : isnotnull(i_item_sk#28)
(26) Filter [codegen id : 9]
Input [2]: [i_item_sk#21, i_product_name#22]
Condition : isnotnull(i_item_sk#21)
(31) BroadcastExchange
Input [2]: [i_item_sk#28, i_product_name#29]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30]
(27) BroadcastExchange
Input [2]: [i_item_sk#21, i_product_name#22]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23]
(32) BroadcastHashJoin [codegen id : 11]
(28) BroadcastHashJoin [codegen id : 11]
Left keys [1]: [item_sk#11]
Right keys [1]: [i_item_sk#28]
Right keys [1]: [i_item_sk#21]
Join condition: None
(33) Project [codegen id : 11]
Output [3]: [rnk#17, item_sk#23, i_product_name#29]
Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#28, i_product_name#29]
(29) Project [codegen id : 11]
Output [3]: [rnk#17, item_sk#18, i_product_name#22]
Input [5]: [item_sk#11, rnk#17, item_sk#18, i_item_sk#21, i_product_name#22]
(34) ReusedExchange [Reuses operator id: 31]
Output [2]: [i_item_sk#31, i_product_name#32]
(30) ReusedExchange [Reuses operator id: 27]
Output [2]: [i_item_sk#24, i_product_name#25]
(35) BroadcastHashJoin [codegen id : 11]
Left keys [1]: [item_sk#23]
Right keys [1]: [i_item_sk#31]
(31) BroadcastHashJoin [codegen id : 11]
Left keys [1]: [item_sk#18]
Right keys [1]: [i_item_sk#24]
Join condition: None
(36) Project [codegen id : 11]
Output [3]: [rnk#17, i_product_name#29 AS best_performing#33, i_product_name#32 AS worst_performing#34]
Input [5]: [rnk#17, item_sk#23, i_product_name#29, i_item_sk#31, i_product_name#32]
(32) Project [codegen id : 11]
Output [3]: [rnk#17, i_product_name#22 AS best_performing#26, i_product_name#25 AS worst_performing#27]
Input [5]: [rnk#17, item_sk#18, i_product_name#22, i_item_sk#24, i_product_name#25]
(37) TakeOrderedAndProject
Input [3]: [rnk#17, best_performing#33, worst_performing#34]
Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#33, worst_performing#34]
(33) TakeOrderedAndProject
Input [3]: [rnk#17, best_performing#26, worst_performing#27]
Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#26, worst_performing#27]
===== Subqueries =====
Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#14, [id=#15]
* HashAggregate (44)
+- Exchange (43)
+- * HashAggregate (42)
+- * Project (41)
+- * Filter (40)
+- * ColumnarToRow (39)
+- Scan parquet default.store_sales (38)
* HashAggregate (40)
+- Exchange (39)
+- * HashAggregate (38)
+- * Project (37)
+- * Filter (36)
+- * ColumnarToRow (35)
+- Scan parquet default.store_sales (34)
(38) Scan parquet default.store_sales
Output [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38]
(34) Scan parquet default.store_sales
Output [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]
Batched: true
Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)]
ReadSchema: struct<ss_addr_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)>
(39) ColumnarToRow [codegen id : 1]
Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38]
(35) ColumnarToRow [codegen id : 1]
Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]
(40) Filter [codegen id : 1]
Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38]
Condition : ((isnotnull(ss_store_sk#36) AND (ss_store_sk#36 = 4)) AND isnull(ss_addr_sk#35))
(36) Filter [codegen id : 1]
Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]
Condition : ((isnotnull(ss_store_sk#29) AND (ss_store_sk#29 = 4)) AND isnull(ss_addr_sk#28))
(41) Project [codegen id : 1]
Output [2]: [ss_store_sk#36, ss_net_profit#37]
Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38]
(37) Project [codegen id : 1]
Output [2]: [ss_store_sk#29, ss_net_profit#30]
Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]
(42) HashAggregate [codegen id : 1]
Input [2]: [ss_store_sk#36, ss_net_profit#37]
Keys [1]: [ss_store_sk#36]
Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#37))]
Aggregate Attributes [2]: [sum#39, count#40]
Results [3]: [ss_store_sk#36, sum#41, count#42]
(38) HashAggregate [codegen id : 1]
Input [2]: [ss_store_sk#29, ss_net_profit#30]
Keys [1]: [ss_store_sk#29]
Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#30))]
Aggregate Attributes [2]: [sum#32, count#33]
Results [3]: [ss_store_sk#29, sum#34, count#35]
(43) Exchange
Input [3]: [ss_store_sk#36, sum#41, count#42]
Arguments: hashpartitioning(ss_store_sk#36, 5), ENSURE_REQUIREMENTS, [id=#43]
(39) Exchange
Input [3]: [ss_store_sk#29, sum#34, count#35]
Arguments: hashpartitioning(ss_store_sk#29, 5), ENSURE_REQUIREMENTS, [id=#36]
(44) HashAggregate [codegen id : 2]
Input [3]: [ss_store_sk#36, sum#41, count#42]
Keys [1]: [ss_store_sk#36]
Functions [1]: [avg(UnscaledValue(ss_net_profit#37))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#37))#44]
Results [1]: [cast((avg(UnscaledValue(ss_net_profit#37))#44 / 100.0) as decimal(11,6)) AS rank_col#45]
Subquery:2 Hosting operator id = 18 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15]
(40) HashAggregate [codegen id : 2]
Input [3]: [ss_store_sk#29, sum#34, count#35]
Keys [1]: [ss_store_sk#29]
Functions [1]: [avg(UnscaledValue(ss_net_profit#30))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#30))#37]
Results [1]: [cast((avg(UnscaledValue(ss_net_profit#30))#37 / 100.0) as decimal(11,6)) AS rank_col#38]

View file

@ -52,20 +52,13 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (7)
Sort [rank_col]
InputAdapter
Exchange #4
WholeStageCodegen (6)
Project [item_sk,rank_col]
Filter [avg(ss_net_profit)]
ReusedSubquery [rank_col] #1
HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count]
InputAdapter
ReusedExchange [ss_item_sk,sum,count] #2
ReusedExchange [item_sk,rank_col] #1
InputAdapter
BroadcastExchange #5
BroadcastExchange #4
WholeStageCodegen (9)
Filter [i_item_sk]
ColumnarToRow
InputAdapter
Scan parquet default.item [i_item_sk,i_product_name]
InputAdapter
ReusedExchange [i_item_sk,i_product_name] #5
ReusedExchange [i_item_sk,i_product_name] #4

View file

@ -1,9 +1,9 @@
== Physical Plan ==
TakeOrderedAndProject (79)
+- * Project (78)
+- * BroadcastHashJoin Inner BuildRight (77)
:- * Project (52)
: +- * BroadcastHashJoin Inner BuildRight (51)
TakeOrderedAndProject (59)
+- * Project (58)
+- * BroadcastHashJoin Inner BuildRight (57)
:- * Project (42)
: +- * BroadcastHashJoin Inner BuildRight (41)
: :- * Filter (26)
: : +- * HashAggregate (25)
: : +- Exchange (24)
@ -30,54 +30,34 @@ TakeOrderedAndProject (79)
: : +- * Filter (19)
: : +- * ColumnarToRow (18)
: : +- Scan parquet default.item (17)
: +- BroadcastExchange (50)
: +- * Filter (49)
: +- * HashAggregate (48)
: +- Exchange (47)
: +- * HashAggregate (46)
: +- * Project (45)
: +- * BroadcastHashJoin Inner BuildRight (44)
: :- * Project (42)
: : +- * BroadcastHashJoin Inner BuildRight (41)
: +- BroadcastExchange (40)
: +- * Filter (39)
: +- * HashAggregate (38)
: +- Exchange (37)
: +- * HashAggregate (36)
: +- * Project (35)
: +- * BroadcastHashJoin Inner BuildRight (34)
: :- * Project (32)
: : +- * BroadcastHashJoin Inner BuildRight (31)
: : :- * Filter (29)
: : : +- * ColumnarToRow (28)
: : : +- Scan parquet default.catalog_sales (27)
: : +- BroadcastExchange (40)
: : +- * Project (39)
: : +- * BroadcastHashJoin LeftSemi BuildRight (38)
: : :- * Filter (32)
: : : +- * ColumnarToRow (31)
: : : +- Scan parquet default.date_dim (30)
: : +- BroadcastExchange (37)
: : +- * Project (36)
: : +- * Filter (35)
: : +- * ColumnarToRow (34)
: : +- Scan parquet default.date_dim (33)
: +- ReusedExchange (43)
+- BroadcastExchange (76)
+- * Filter (75)
+- * HashAggregate (74)
+- Exchange (73)
+- * HashAggregate (72)
+- * Project (71)
+- * BroadcastHashJoin Inner BuildRight (70)
:- * Project (68)
: +- * BroadcastHashJoin Inner BuildRight (67)
: :- * Filter (55)
: : +- * ColumnarToRow (54)
: : +- Scan parquet default.web_sales (53)
: +- BroadcastExchange (66)
: +- * Project (65)
: +- * BroadcastHashJoin LeftSemi BuildRight (64)
: :- * Filter (58)
: : +- * ColumnarToRow (57)
: : +- Scan parquet default.date_dim (56)
: +- BroadcastExchange (63)
: +- * Project (62)
: +- * Filter (61)
: +- * ColumnarToRow (60)
: +- Scan parquet default.date_dim (59)
+- ReusedExchange (69)
: : +- ReusedExchange (30)
: +- ReusedExchange (33)
+- BroadcastExchange (56)
+- * Filter (55)
+- * HashAggregate (54)
+- Exchange (53)
+- * HashAggregate (52)
+- * Project (51)
+- * BroadcastHashJoin Inner BuildRight (50)
:- * Project (48)
: +- * BroadcastHashJoin Inner BuildRight (47)
: :- * Filter (45)
: : +- * ColumnarToRow (44)
: : +- Scan parquet default.web_sales (43)
: +- ReusedExchange (46)
+- ReusedExchange (49)
(1) Scan parquet default.store_sales
@ -217,264 +197,168 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23]
Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23]
Condition : isnotnull(cs_item_sk#21)
(30) Scan parquet default.date_dim
Output [2]: [d_date_sk#24, d_date#25]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_date:date>
(31) ColumnarToRow [codegen id : 6]
Input [2]: [d_date_sk#24, d_date#25]
(32) Filter [codegen id : 6]
Input [2]: [d_date_sk#24, d_date#25]
Condition : isnotnull(d_date_sk#24)
(33) Scan parquet default.date_dim
Output [2]: [d_date#26, d_week_seq#27]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq)]
ReadSchema: struct<d_date:date,d_week_seq:int>
(34) ColumnarToRow [codegen id : 5]
Input [2]: [d_date#26, d_week_seq#27]
(35) Filter [codegen id : 5]
Input [2]: [d_date#26, d_week_seq#27]
Condition : (isnotnull(d_week_seq#27) AND (d_week_seq#27 = ReusedSubquery Subquery scalar-subquery#8, [id=#9]))
(36) Project [codegen id : 5]
Output [1]: [d_date#26]
Input [2]: [d_date#26, d_week_seq#27]
(37) BroadcastExchange
Input [1]: [d_date#26]
Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#28]
(38) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [d_date#25]
Right keys [1]: [d_date#26]
Join condition: None
(39) Project [codegen id : 6]
(30) ReusedExchange [Reuses operator id: 14]
Output [1]: [d_date_sk#24]
Input [2]: [d_date_sk#24, d_date#25]
(40) BroadcastExchange
Input [1]: [d_date_sk#24]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29]
(41) BroadcastHashJoin [codegen id : 8]
(31) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [cs_sold_date_sk#23]
Right keys [1]: [d_date_sk#24]
Join condition: None
(42) Project [codegen id : 8]
(32) Project [codegen id : 8]
Output [2]: [cs_item_sk#21, cs_ext_sales_price#22]
Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24]
(43) ReusedExchange [Reuses operator id: 20]
Output [2]: [i_item_sk#30, i_item_id#31]
(33) ReusedExchange [Reuses operator id: 20]
Output [2]: [i_item_sk#25, i_item_id#26]
(44) BroadcastHashJoin [codegen id : 8]
(34) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [cs_item_sk#21]
Right keys [1]: [i_item_sk#30]
Right keys [1]: [i_item_sk#25]
Join condition: None
(45) Project [codegen id : 8]
Output [2]: [cs_ext_sales_price#22, i_item_id#31]
Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#30, i_item_id#31]
(35) Project [codegen id : 8]
Output [2]: [cs_ext_sales_price#22, i_item_id#26]
Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#25, i_item_id#26]
(46) HashAggregate [codegen id : 8]
Input [2]: [cs_ext_sales_price#22, i_item_id#31]
Keys [1]: [i_item_id#31]
(36) HashAggregate [codegen id : 8]
Input [2]: [cs_ext_sales_price#22, i_item_id#26]
Keys [1]: [i_item_id#26]
Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))]
Aggregate Attributes [1]: [sum#32]
Results [2]: [i_item_id#31, sum#33]
Aggregate Attributes [1]: [sum#27]
Results [2]: [i_item_id#26, sum#28]
(47) Exchange
Input [2]: [i_item_id#31, sum#33]
Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34]
(37) Exchange
Input [2]: [i_item_id#26, sum#28]
Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#29]
(48) HashAggregate [codegen id : 9]
Input [2]: [i_item_id#31, sum#33]
Keys [1]: [i_item_id#31]
(38) HashAggregate [codegen id : 9]
Input [2]: [i_item_id#26, sum#28]
Keys [1]: [i_item_id#26]
Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))]
Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35]
Results [2]: [i_item_id#31 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37]
Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30]
Results [2]: [i_item_id#26 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32]
(49) Filter [codegen id : 9]
Input [2]: [item_id#36, cs_item_rev#37]
Condition : isnotnull(cs_item_rev#37)
(39) Filter [codegen id : 9]
Input [2]: [item_id#31, cs_item_rev#32]
Condition : isnotnull(cs_item_rev#32)
(50) BroadcastExchange
Input [2]: [item_id#36, cs_item_rev#37]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38]
(40) BroadcastExchange
Input [2]: [item_id#31, cs_item_rev#32]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33]
(51) BroadcastHashJoin [codegen id : 15]
(41) BroadcastHashJoin [codegen id : 15]
Left keys [1]: [item_id#19]
Right keys [1]: [item_id#36]
Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true)))
Right keys [1]: [item_id#31]
Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true)))
(52) Project [codegen id : 15]
Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37]
Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37]
(42) Project [codegen id : 15]
Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32]
Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32]
(53) Scan parquet default.web_sales
Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41]
(43) Scan parquet default.web_sales
Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)]
PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)]
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int,ws_ext_sales_price:decimal(7,2)>
(54) ColumnarToRow [codegen id : 13]
Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41]
(44) ColumnarToRow [codegen id : 13]
Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36]
(55) Filter [codegen id : 13]
Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41]
Condition : isnotnull(ws_item_sk#39)
(45) Filter [codegen id : 13]
Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36]
Condition : isnotnull(ws_item_sk#34)
(56) Scan parquet default.date_dim
Output [2]: [d_date_sk#42, d_date#43]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_date:date>
(46) ReusedExchange [Reuses operator id: 14]
Output [1]: [d_date_sk#37]
(57) ColumnarToRow [codegen id : 11]
Input [2]: [d_date_sk#42, d_date#43]
(58) Filter [codegen id : 11]
Input [2]: [d_date_sk#42, d_date#43]
Condition : isnotnull(d_date_sk#42)
(59) Scan parquet default.date_dim
Output [2]: [d_date#44, d_week_seq#45]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq)]
ReadSchema: struct<d_date:date,d_week_seq:int>
(60) ColumnarToRow [codegen id : 10]
Input [2]: [d_date#44, d_week_seq#45]
(61) Filter [codegen id : 10]
Input [2]: [d_date#44, d_week_seq#45]
Condition : (isnotnull(d_week_seq#45) AND (d_week_seq#45 = ReusedSubquery Subquery scalar-subquery#8, [id=#9]))
(62) Project [codegen id : 10]
Output [1]: [d_date#44]
Input [2]: [d_date#44, d_week_seq#45]
(63) BroadcastExchange
Input [1]: [d_date#44]
Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#46]
(64) BroadcastHashJoin [codegen id : 11]
Left keys [1]: [d_date#43]
Right keys [1]: [d_date#44]
(47) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_sold_date_sk#36]
Right keys [1]: [d_date_sk#37]
Join condition: None
(65) Project [codegen id : 11]
Output [1]: [d_date_sk#42]
Input [2]: [d_date_sk#42, d_date#43]
(48) Project [codegen id : 13]
Output [2]: [ws_item_sk#34, ws_ext_sales_price#35]
Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#37]
(66) BroadcastExchange
Input [1]: [d_date_sk#42]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47]
(49) ReusedExchange [Reuses operator id: 20]
Output [2]: [i_item_sk#38, i_item_id#39]
(67) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_sold_date_sk#41]
Right keys [1]: [d_date_sk#42]
(50) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_item_sk#34]
Right keys [1]: [i_item_sk#38]
Join condition: None
(68) Project [codegen id : 13]
Output [2]: [ws_item_sk#39, ws_ext_sales_price#40]
Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, d_date_sk#42]
(51) Project [codegen id : 13]
Output [2]: [ws_ext_sales_price#35, i_item_id#39]
Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#38, i_item_id#39]
(69) ReusedExchange [Reuses operator id: 20]
Output [2]: [i_item_sk#48, i_item_id#49]
(52) HashAggregate [codegen id : 13]
Input [2]: [ws_ext_sales_price#35, i_item_id#39]
Keys [1]: [i_item_id#39]
Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))]
Aggregate Attributes [1]: [sum#40]
Results [2]: [i_item_id#39, sum#41]
(70) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_item_sk#39]
Right keys [1]: [i_item_sk#48]
Join condition: None
(53) Exchange
Input [2]: [i_item_id#39, sum#41]
Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#42]
(71) Project [codegen id : 13]
Output [2]: [ws_ext_sales_price#40, i_item_id#49]
Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, i_item_sk#48, i_item_id#49]
(54) HashAggregate [codegen id : 14]
Input [2]: [i_item_id#39, sum#41]
Keys [1]: [i_item_id#39]
Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))]
Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43]
Results [2]: [i_item_id#39 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45]
(72) HashAggregate [codegen id : 13]
Input [2]: [ws_ext_sales_price#40, i_item_id#49]
Keys [1]: [i_item_id#49]
Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))]
Aggregate Attributes [1]: [sum#50]
Results [2]: [i_item_id#49, sum#51]
(55) Filter [codegen id : 14]
Input [2]: [item_id#44, ws_item_rev#45]
Condition : isnotnull(ws_item_rev#45)
(73) Exchange
Input [2]: [i_item_id#49, sum#51]
Arguments: hashpartitioning(i_item_id#49, 5), ENSURE_REQUIREMENTS, [id=#52]
(56) BroadcastExchange
Input [2]: [item_id#44, ws_item_rev#45]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46]
(74) HashAggregate [codegen id : 14]
Input [2]: [i_item_id#49, sum#51]
Keys [1]: [i_item_id#49]
Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))]
Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53]
Results [2]: [i_item_id#49 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55]
(75) Filter [codegen id : 14]
Input [2]: [item_id#54, ws_item_rev#55]
Condition : isnotnull(ws_item_rev#55)
(76) BroadcastExchange
Input [2]: [item_id#54, ws_item_rev#55]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56]
(77) BroadcastHashJoin [codegen id : 15]
(57) BroadcastHashJoin [codegen id : 15]
Left keys [1]: [item_id#19]
Right keys [1]: [item_id#54]
Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true)))
Right keys [1]: [item_id#44]
Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true)))
(78) Project [codegen id : 15]
Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60]
Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55]
(58) Project [codegen id : 15]
Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50]
Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45]
(79) TakeOrderedAndProject
Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60]
Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60]
(59) TakeOrderedAndProject
Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50]
Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50]
===== Subqueries =====
Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9]
* Project (83)
+- * Filter (82)
+- * ColumnarToRow (81)
+- Scan parquet default.date_dim (80)
* Project (63)
+- * Filter (62)
+- * ColumnarToRow (61)
+- Scan parquet default.date_dim (60)
(80) Scan parquet default.date_dim
Output [2]: [d_date#61, d_week_seq#62]
(60) Scan parquet default.date_dim
Output [2]: [d_date#51, d_week_seq#52]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)]
ReadSchema: struct<d_date:date,d_week_seq:int>
(81) ColumnarToRow [codegen id : 1]
Input [2]: [d_date#61, d_week_seq#62]
(61) ColumnarToRow [codegen id : 1]
Input [2]: [d_date#51, d_week_seq#52]
(82) Filter [codegen id : 1]
Input [2]: [d_date#61, d_week_seq#62]
Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03))
(62) Filter [codegen id : 1]
Input [2]: [d_date#51, d_week_seq#52]
Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03))
(83) Project [codegen id : 1]
Output [1]: [d_week_seq#62]
Input [2]: [d_date#61, d_week_seq#62]
Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9]
Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9]
(63) Project [codegen id : 1]
Output [1]: [d_week_seq#52]
Input [2]: [d_date#51, d_week_seq#52]

View file

@ -67,32 +67,16 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
InputAdapter
Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
Project [d_date_sk]
BroadcastHashJoin [d_date,d_date]
Filter [d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (5)
Project [d_date]
Filter [d_week_seq]
ReusedSubquery [d_week_seq] #1
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date,d_week_seq]
ReusedExchange [d_date_sk] #2
InputAdapter
ReusedExchange [i_item_sk,i_item_id] #4
InputAdapter
BroadcastExchange #9
BroadcastExchange #7
WholeStageCodegen (14)
Filter [ws_item_rev]
HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum]
InputAdapter
Exchange [i_item_id] #10
Exchange [i_item_id] #8
WholeStageCodegen (13)
HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum]
Project [ws_ext_sales_price,i_item_id]
@ -104,22 +88,6 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
InputAdapter
Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (11)
Project [d_date_sk]
BroadcastHashJoin [d_date,d_date]
Filter [d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
WholeStageCodegen (10)
Project [d_date]
Filter [d_week_seq]
ReusedSubquery [d_week_seq] #1
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date,d_week_seq]
ReusedExchange [d_date_sk] #2
InputAdapter
ReusedExchange [i_item_sk,i_item_id] #4

View file

@ -1,9 +1,9 @@
== Physical Plan ==
TakeOrderedAndProject (79)
+- * Project (78)
+- * BroadcastHashJoin Inner BuildRight (77)
:- * Project (52)
: +- * BroadcastHashJoin Inner BuildRight (51)
TakeOrderedAndProject (59)
+- * Project (58)
+- * BroadcastHashJoin Inner BuildRight (57)
:- * Project (42)
: +- * BroadcastHashJoin Inner BuildRight (41)
: :- * Filter (26)
: : +- * HashAggregate (25)
: : +- Exchange (24)
@ -30,54 +30,34 @@ TakeOrderedAndProject (79)
: : +- * Filter (15)
: : +- * ColumnarToRow (14)
: : +- Scan parquet default.date_dim (13)
: +- BroadcastExchange (50)
: +- * Filter (49)
: +- * HashAggregate (48)
: +- Exchange (47)
: +- * HashAggregate (46)
: +- * Project (45)
: +- * BroadcastHashJoin Inner BuildRight (44)
: +- BroadcastExchange (40)
: +- * Filter (39)
: +- * HashAggregate (38)
: +- Exchange (37)
: +- * HashAggregate (36)
: +- * Project (35)
: +- * BroadcastHashJoin Inner BuildRight (34)
: :- * Project (32)
: : +- * BroadcastHashJoin Inner BuildRight (31)
: : :- * Filter (29)
: : : +- * ColumnarToRow (28)
: : : +- Scan parquet default.catalog_sales (27)
: : +- ReusedExchange (30)
: +- BroadcastExchange (43)
: +- * Project (42)
: +- * BroadcastHashJoin LeftSemi BuildRight (41)
: :- * Filter (35)
: : +- * ColumnarToRow (34)
: : +- Scan parquet default.date_dim (33)
: +- BroadcastExchange (40)
: +- * Project (39)
: +- * Filter (38)
: +- * ColumnarToRow (37)
: +- Scan parquet default.date_dim (36)
+- BroadcastExchange (76)
+- * Filter (75)
+- * HashAggregate (74)
+- Exchange (73)
+- * HashAggregate (72)
+- * Project (71)
+- * BroadcastHashJoin Inner BuildRight (70)
:- * Project (58)
: +- * BroadcastHashJoin Inner BuildRight (57)
: :- * Filter (55)
: : +- * ColumnarToRow (54)
: : +- Scan parquet default.web_sales (53)
: +- ReusedExchange (56)
+- BroadcastExchange (69)
+- * Project (68)
+- * BroadcastHashJoin LeftSemi BuildRight (67)
:- * Filter (61)
: +- * ColumnarToRow (60)
: +- Scan parquet default.date_dim (59)
+- BroadcastExchange (66)
+- * Project (65)
+- * Filter (64)
+- * ColumnarToRow (63)
+- Scan parquet default.date_dim (62)
: +- ReusedExchange (33)
+- BroadcastExchange (56)
+- * Filter (55)
+- * HashAggregate (54)
+- Exchange (53)
+- * HashAggregate (52)
+- * Project (51)
+- * BroadcastHashJoin Inner BuildRight (50)
:- * Project (48)
: +- * BroadcastHashJoin Inner BuildRight (47)
: :- * Filter (45)
: : +- * ColumnarToRow (44)
: : +- Scan parquet default.web_sales (43)
: +- ReusedExchange (46)
+- ReusedExchange (49)
(1) Scan parquet default.store_sales
@ -229,252 +209,156 @@ Join condition: None
Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25]
Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25]
(33) Scan parquet default.date_dim
Output [2]: [d_date_sk#26, d_date#27]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_date:date>
(34) ColumnarToRow [codegen id : 7]
Input [2]: [d_date_sk#26, d_date#27]
(35) Filter [codegen id : 7]
Input [2]: [d_date_sk#26, d_date#27]
Condition : isnotnull(d_date_sk#26)
(36) Scan parquet default.date_dim
Output [2]: [d_date#28, d_week_seq#29]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq)]
ReadSchema: struct<d_date:date,d_week_seq:int>
(37) ColumnarToRow [codegen id : 6]
Input [2]: [d_date#28, d_week_seq#29]
(38) Filter [codegen id : 6]
Input [2]: [d_date#28, d_week_seq#29]
Condition : (isnotnull(d_week_seq#29) AND (d_week_seq#29 = ReusedSubquery Subquery scalar-subquery#11, [id=#12]))
(39) Project [codegen id : 6]
Output [1]: [d_date#28]
Input [2]: [d_date#28, d_week_seq#29]
(40) BroadcastExchange
Input [1]: [d_date#28]
Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#30]
(41) BroadcastHashJoin [codegen id : 7]
Left keys [1]: [d_date#27]
Right keys [1]: [d_date#28]
Join condition: None
(42) Project [codegen id : 7]
(33) ReusedExchange [Reuses operator id: 20]
Output [1]: [d_date_sk#26]
Input [2]: [d_date_sk#26, d_date#27]
(43) BroadcastExchange
Input [1]: [d_date_sk#26]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31]
(44) BroadcastHashJoin [codegen id : 8]
(34) BroadcastHashJoin [codegen id : 8]
Left keys [1]: [cs_sold_date_sk#23]
Right keys [1]: [d_date_sk#26]
Join condition: None
(45) Project [codegen id : 8]
(35) Project [codegen id : 8]
Output [2]: [cs_ext_sales_price#22, i_item_id#25]
Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26]
(46) HashAggregate [codegen id : 8]
(36) HashAggregate [codegen id : 8]
Input [2]: [cs_ext_sales_price#22, i_item_id#25]
Keys [1]: [i_item_id#25]
Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))]
Aggregate Attributes [1]: [sum#32]
Results [2]: [i_item_id#25, sum#33]
Aggregate Attributes [1]: [sum#27]
Results [2]: [i_item_id#25, sum#28]
(47) Exchange
Input [2]: [i_item_id#25, sum#33]
Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#34]
(37) Exchange
Input [2]: [i_item_id#25, sum#28]
Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#29]
(48) HashAggregate [codegen id : 9]
Input [2]: [i_item_id#25, sum#33]
(38) HashAggregate [codegen id : 9]
Input [2]: [i_item_id#25, sum#28]
Keys [1]: [i_item_id#25]
Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))]
Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35]
Results [2]: [i_item_id#25 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37]
Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30]
Results [2]: [i_item_id#25 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32]
(49) Filter [codegen id : 9]
Input [2]: [item_id#36, cs_item_rev#37]
Condition : isnotnull(cs_item_rev#37)
(39) Filter [codegen id : 9]
Input [2]: [item_id#31, cs_item_rev#32]
Condition : isnotnull(cs_item_rev#32)
(50) BroadcastExchange
Input [2]: [item_id#36, cs_item_rev#37]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38]
(40) BroadcastExchange
Input [2]: [item_id#31, cs_item_rev#32]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33]
(51) BroadcastHashJoin [codegen id : 15]
(41) BroadcastHashJoin [codegen id : 15]
Left keys [1]: [item_id#19]
Right keys [1]: [item_id#36]
Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true)))
Right keys [1]: [item_id#31]
Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true)))
(52) Project [codegen id : 15]
Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37]
Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37]
(42) Project [codegen id : 15]
Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32]
Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32]
(53) Scan parquet default.web_sales
Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41]
(43) Scan parquet default.web_sales
Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)]
PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)]
PushedFilters: [IsNotNull(ws_item_sk)]
ReadSchema: struct<ws_item_sk:int,ws_ext_sales_price:decimal(7,2)>
(54) ColumnarToRow [codegen id : 13]
Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41]
(44) ColumnarToRow [codegen id : 13]
Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36]
(55) Filter [codegen id : 13]
Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41]
Condition : isnotnull(ws_item_sk#39)
(45) Filter [codegen id : 13]
Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36]
Condition : isnotnull(ws_item_sk#34)
(56) ReusedExchange [Reuses operator id: 7]
Output [2]: [i_item_sk#42, i_item_id#43]
(46) ReusedExchange [Reuses operator id: 7]
Output [2]: [i_item_sk#37, i_item_id#38]
(57) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_item_sk#39]
Right keys [1]: [i_item_sk#42]
(47) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_item_sk#34]
Right keys [1]: [i_item_sk#37]
Join condition: None
(58) Project [codegen id : 13]
Output [3]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43]
Input [5]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_sk#42, i_item_id#43]
(48) Project [codegen id : 13]
Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38]
Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#37, i_item_id#38]
(59) Scan parquet default.date_dim
Output [2]: [d_date_sk#44, d_date#45]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_date:date>
(49) ReusedExchange [Reuses operator id: 20]
Output [1]: [d_date_sk#39]
(60) ColumnarToRow [codegen id : 12]
Input [2]: [d_date_sk#44, d_date#45]
(61) Filter [codegen id : 12]
Input [2]: [d_date_sk#44, d_date#45]
Condition : isnotnull(d_date_sk#44)
(62) Scan parquet default.date_dim
Output [2]: [d_date#46, d_week_seq#47]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq)]
ReadSchema: struct<d_date:date,d_week_seq:int>
(63) ColumnarToRow [codegen id : 11]
Input [2]: [d_date#46, d_week_seq#47]
(64) Filter [codegen id : 11]
Input [2]: [d_date#46, d_week_seq#47]
Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = ReusedSubquery Subquery scalar-subquery#11, [id=#12]))
(65) Project [codegen id : 11]
Output [1]: [d_date#46]
Input [2]: [d_date#46, d_week_seq#47]
(66) BroadcastExchange
Input [1]: [d_date#46]
Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#48]
(67) BroadcastHashJoin [codegen id : 12]
Left keys [1]: [d_date#45]
Right keys [1]: [d_date#46]
(50) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_sold_date_sk#36]
Right keys [1]: [d_date_sk#39]
Join condition: None
(68) Project [codegen id : 12]
Output [1]: [d_date_sk#44]
Input [2]: [d_date_sk#44, d_date#45]
(51) Project [codegen id : 13]
Output [2]: [ws_ext_sales_price#35, i_item_id#38]
Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#38, d_date_sk#39]
(69) BroadcastExchange
Input [1]: [d_date_sk#44]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49]
(52) HashAggregate [codegen id : 13]
Input [2]: [ws_ext_sales_price#35, i_item_id#38]
Keys [1]: [i_item_id#38]
Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))]
Aggregate Attributes [1]: [sum#40]
Results [2]: [i_item_id#38, sum#41]
(70) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [ws_sold_date_sk#41]
Right keys [1]: [d_date_sk#44]
Join condition: None
(53) Exchange
Input [2]: [i_item_id#38, sum#41]
Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [id=#42]
(71) Project [codegen id : 13]
Output [2]: [ws_ext_sales_price#40, i_item_id#43]
Input [4]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43, d_date_sk#44]
(54) HashAggregate [codegen id : 14]
Input [2]: [i_item_id#38, sum#41]
Keys [1]: [i_item_id#38]
Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))]
Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43]
Results [2]: [i_item_id#38 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45]
(72) HashAggregate [codegen id : 13]
Input [2]: [ws_ext_sales_price#40, i_item_id#43]
Keys [1]: [i_item_id#43]
Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))]
Aggregate Attributes [1]: [sum#50]
Results [2]: [i_item_id#43, sum#51]
(55) Filter [codegen id : 14]
Input [2]: [item_id#44, ws_item_rev#45]
Condition : isnotnull(ws_item_rev#45)
(73) Exchange
Input [2]: [i_item_id#43, sum#51]
Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#52]
(56) BroadcastExchange
Input [2]: [item_id#44, ws_item_rev#45]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46]
(74) HashAggregate [codegen id : 14]
Input [2]: [i_item_id#43, sum#51]
Keys [1]: [i_item_id#43]
Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))]
Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53]
Results [2]: [i_item_id#43 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55]
(75) Filter [codegen id : 14]
Input [2]: [item_id#54, ws_item_rev#55]
Condition : isnotnull(ws_item_rev#55)
(76) BroadcastExchange
Input [2]: [item_id#54, ws_item_rev#55]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56]
(77) BroadcastHashJoin [codegen id : 15]
(57) BroadcastHashJoin [codegen id : 15]
Left keys [1]: [item_id#19]
Right keys [1]: [item_id#54]
Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true)))
Right keys [1]: [item_id#44]
Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true)))
(78) Project [codegen id : 15]
Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60]
Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55]
(58) Project [codegen id : 15]
Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50]
Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45]
(79) TakeOrderedAndProject
Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60]
Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60]
(59) TakeOrderedAndProject
Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50]
Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50]
===== Subqueries =====
Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12]
* Project (83)
+- * Filter (82)
+- * ColumnarToRow (81)
+- Scan parquet default.date_dim (80)
* Project (63)
+- * Filter (62)
+- * ColumnarToRow (61)
+- Scan parquet default.date_dim (60)
(80) Scan parquet default.date_dim
Output [2]: [d_date#61, d_week_seq#62]
(60) Scan parquet default.date_dim
Output [2]: [d_date#51, d_week_seq#52]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)]
ReadSchema: struct<d_date:date,d_week_seq:int>
(81) ColumnarToRow [codegen id : 1]
Input [2]: [d_date#61, d_week_seq#62]
(61) ColumnarToRow [codegen id : 1]
Input [2]: [d_date#51, d_week_seq#52]
(82) Filter [codegen id : 1]
Input [2]: [d_date#61, d_week_seq#62]
Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03))
(62) Filter [codegen id : 1]
Input [2]: [d_date#51, d_week_seq#52]
Condition : (isnotnull(d_date#51) AND (d_date#51 = 2000-01-03))
(83) Project [codegen id : 1]
Output [1]: [d_week_seq#62]
Input [2]: [d_date#61, d_week_seq#62]
Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12]
Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12]
(63) Project [codegen id : 1]
Output [1]: [d_week_seq#52]
Input [2]: [d_date#51, d_week_seq#52]

View file

@ -69,30 +69,14 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
InputAdapter
ReusedExchange [i_item_sk,i_item_id] #2
InputAdapter
BroadcastExchange #7
WholeStageCodegen (7)
Project [d_date_sk]
BroadcastHashJoin [d_date,d_date]
Filter [d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (6)
Project [d_date]
Filter [d_week_seq]
ReusedSubquery [d_week_seq] #1
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date,d_week_seq]
ReusedExchange [d_date_sk] #3
InputAdapter
BroadcastExchange #9
BroadcastExchange #7
WholeStageCodegen (14)
Filter [ws_item_rev]
HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum]
InputAdapter
Exchange [i_item_id] #10
Exchange [i_item_id] #8
WholeStageCodegen (13)
HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum]
Project [ws_ext_sales_price,i_item_id]
@ -106,20 +90,4 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
InputAdapter
ReusedExchange [i_item_sk,i_item_id] #2
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
Project [d_date_sk]
BroadcastHashJoin [d_date,d_date]
Filter [d_date_sk]
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
WholeStageCodegen (11)
Project [d_date]
Filter [d_week_seq]
ReusedSubquery [d_week_seq] #1
ColumnarToRow
InputAdapter
Scan parquet default.date_dim [d_date,d_week_seq]
ReusedExchange [d_date_sk] #3

View file

@ -1,9 +1,9 @@
== Physical Plan ==
TakeOrderedAndProject (213)
+- * HashAggregate (212)
+- Exchange (211)
+- * HashAggregate (210)
+- Union (209)
TakeOrderedAndProject (157)
+- * HashAggregate (156)
+- Exchange (155)
+- * HashAggregate (154)
+- Union (153)
:- * HashAggregate (132)
: +- Exchange (131)
: +- * HashAggregate (130)
@ -136,82 +136,26 @@ TakeOrderedAndProject (213)
: : : +- ReusedExchange (115)
: : +- ReusedExchange (118)
: +- ReusedExchange (121)
:- * HashAggregate (151)
: +- Exchange (150)
: +- * HashAggregate (149)
: +- * HashAggregate (148)
: +- Exchange (147)
: +- * HashAggregate (146)
: +- Union (145)
: :- * Project (136)
: : +- * Filter (135)
: : +- * HashAggregate (134)
: : +- ReusedExchange (133)
: :- * Project (140)
: : +- * Filter (139)
: : +- * HashAggregate (138)
: : +- ReusedExchange (137)
: +- * Project (144)
: +- * Filter (143)
: +- * HashAggregate (142)
: +- ReusedExchange (141)
:- * HashAggregate (170)
: +- Exchange (169)
: +- * HashAggregate (168)
: +- * HashAggregate (167)
: +- Exchange (166)
: +- * HashAggregate (165)
: +- Union (164)
: :- * Project (155)
: : +- * Filter (154)
: : +- * HashAggregate (153)
: : +- ReusedExchange (152)
: :- * Project (159)
: : +- * Filter (158)
: : +- * HashAggregate (157)
: : +- ReusedExchange (156)
: +- * Project (163)
: +- * Filter (162)
: +- * HashAggregate (161)
: +- ReusedExchange (160)
:- * HashAggregate (189)
: +- Exchange (188)
: +- * HashAggregate (187)
: +- * HashAggregate (186)
: +- Exchange (185)
: +- * HashAggregate (184)
: +- Union (183)
: :- * Project (174)
: : +- * Filter (173)
: : +- * HashAggregate (172)
: : +- ReusedExchange (171)
: :- * Project (178)
: : +- * Filter (177)
: : +- * HashAggregate (176)
: : +- ReusedExchange (175)
: +- * Project (182)
: +- * Filter (181)
: +- * HashAggregate (180)
: +- ReusedExchange (179)
+- * HashAggregate (208)
+- Exchange (207)
+- * HashAggregate (206)
+- * HashAggregate (205)
+- Exchange (204)
+- * HashAggregate (203)
+- Union (202)
:- * Project (193)
: +- * Filter (192)
: +- * HashAggregate (191)
: +- ReusedExchange (190)
:- * Project (197)
: +- * Filter (196)
: +- * HashAggregate (195)
: +- ReusedExchange (194)
+- * Project (201)
+- * Filter (200)
+- * HashAggregate (199)
+- ReusedExchange (198)
:- * HashAggregate (137)
: +- Exchange (136)
: +- * HashAggregate (135)
: +- * HashAggregate (134)
: +- ReusedExchange (133)
:- * HashAggregate (142)
: +- Exchange (141)
: +- * HashAggregate (140)
: +- * HashAggregate (139)
: +- ReusedExchange (138)
:- * HashAggregate (147)
: +- Exchange (146)
: +- * HashAggregate (145)
: +- * HashAggregate (144)
: +- ReusedExchange (143)
+- * HashAggregate (152)
+- Exchange (151)
+- * HashAggregate (150)
+- * HashAggregate (149)
+- ReusedExchange (148)
(1) Scan parquet default.store_sales
@ -816,581 +760,325 @@ Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128]
Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130]
(133) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136]
Output [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, sum#136]
(134) HashAggregate [codegen id : 186]
Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136]
Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140]
Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141]
(135) Filter [codegen id : 186]
Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(136) Project [codegen id : 186]
Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69]
Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141]
(137) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147]
(138) HashAggregate [codegen id : 232]
Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147]
Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151]
Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152]
(139) Filter [codegen id : 232]
Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(140) Project [codegen id : 232]
Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94]
Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152]
(141) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158]
(142) HashAggregate [codegen id : 278]
Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158]
Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162]
Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163]
(143) Filter [codegen id : 278]
Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(144) Project [codegen id : 278]
Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117]
Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163]
(145) Union
(146) HashAggregate [codegen id : 279]
Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69]
Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133]
Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)]
Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166]
Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169]
(147) Exchange
Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169]
Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170]
(148) HashAggregate [codegen id : 280]
Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169]
(134) HashAggregate [codegen id : 280]
Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, sum#136]
Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133]
Functions [2]: [sum(sales#68), sum(number_sales#69)]
Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172]
Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130]
Aggregate Attributes [2]: [sum(sales#68)#137, sum(number_sales#69)#138]
Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#137 AS sum_sales#129, sum(number_sales#69)#138 AS number_sales#130]
(149) HashAggregate [codegen id : 280]
(135) HashAggregate [codegen id : 280]
Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130]
Keys [3]: [channel#73, i_brand_id#131, i_class_id#132]
Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)]
Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175]
Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178]
Aggregate Attributes [3]: [sum#139, isEmpty#140, sum#141]
Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144]
(150) Exchange
Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178]
Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179]
(136) Exchange
Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144]
Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#145]
(151) HashAggregate [codegen id : 281]
Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178]
(137) HashAggregate [codegen id : 281]
Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144]
Keys [3]: [channel#73, i_brand_id#131, i_class_id#132]
Functions [2]: [sum(sum_sales#129), sum(number_sales#130)]
Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181]
Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184]
Aggregate Attributes [2]: [sum(sum_sales#129)#146, sum(number_sales#130)#147]
Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#148, sum(sum_sales#129)#146 AS sum(sum_sales)#149, sum(number_sales#130)#147 AS sum(number_sales)#150]
(152) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190]
(138) ReusedExchange [Reuses operator id: unknown]
Output [7]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156]
(153) HashAggregate [codegen id : 327]
Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190]
Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194]
Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195]
(154) Filter [codegen id : 327]
Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(155) Project [codegen id : 327]
Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69]
Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195]
(156) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201]
(157) HashAggregate [codegen id : 373]
Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201]
Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205]
Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206]
(158) Filter [codegen id : 373]
Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(159) Project [codegen id : 373]
Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94]
Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206]
(160) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212]
(161) HashAggregate [codegen id : 419]
Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212]
Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216]
Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217]
(162) Filter [codegen id : 419]
Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(163) Project [codegen id : 419]
Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117]
Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217]
(164) Union
(165) HashAggregate [codegen id : 420]
Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69]
Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187]
Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)]
Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220]
Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223]
(166) Exchange
Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223]
Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224]
(167) HashAggregate [codegen id : 421]
Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223]
Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187]
(139) HashAggregate [codegen id : 421]
Input [7]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156]
Keys [4]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153]
Functions [2]: [sum(sales#68), sum(number_sales#69)]
Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226]
Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130]
Aggregate Attributes [2]: [sum(sales#68)#157, sum(number_sales#69)#158]
Results [4]: [channel#73, i_brand_id#151, sum(sales#68)#157 AS sum_sales#129, sum(number_sales#69)#158 AS number_sales#130]
(168) HashAggregate [codegen id : 421]
Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130]
Keys [2]: [channel#73, i_brand_id#185]
(140) HashAggregate [codegen id : 421]
Input [4]: [channel#73, i_brand_id#151, sum_sales#129, number_sales#130]
Keys [2]: [channel#73, i_brand_id#151]
Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)]
Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229]
Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232]
Aggregate Attributes [3]: [sum#159, isEmpty#160, sum#161]
Results [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164]
(169) Exchange
Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232]
Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233]
(141) Exchange
Input [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164]
Arguments: hashpartitioning(channel#73, i_brand_id#151, 5), ENSURE_REQUIREMENTS, [id=#165]
(170) HashAggregate [codegen id : 422]
Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232]
Keys [2]: [channel#73, i_brand_id#185]
(142) HashAggregate [codegen id : 422]
Input [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164]
Keys [2]: [channel#73, i_brand_id#151]
Functions [2]: [sum(sum_sales#129), sum(number_sales#130)]
Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235]
Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239]
Aggregate Attributes [2]: [sum(sum_sales#129)#166, sum(number_sales#130)#167]
Results [6]: [channel#73, i_brand_id#151, null AS i_class_id#168, null AS i_category_id#169, sum(sum_sales#129)#166 AS sum(sum_sales)#170, sum(number_sales#130)#167 AS sum(number_sales)#171]
(171) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245]
(143) ReusedExchange [Reuses operator id: unknown]
Output [7]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174, sum#175, isEmpty#176, sum#177]
(172) HashAggregate [codegen id : 468]
Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245]
Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249]
Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250]
(173) Filter [codegen id : 468]
Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(174) Project [codegen id : 468]
Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69]
Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250]
(175) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256]
(176) HashAggregate [codegen id : 514]
Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256]
Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260]
Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261]
(177) Filter [codegen id : 514]
Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(178) Project [codegen id : 514]
Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94]
Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261]
(179) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267]
(180) HashAggregate [codegen id : 560]
Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267]
Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271]
Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272]
(181) Filter [codegen id : 560]
Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(182) Project [codegen id : 560]
Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117]
Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272]
(183) Union
(184) HashAggregate [codegen id : 561]
Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69]
Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242]
Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)]
Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275]
Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278]
(185) Exchange
Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278]
Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279]
(186) HashAggregate [codegen id : 562]
Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278]
Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242]
(144) HashAggregate [codegen id : 562]
Input [7]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174, sum#175, isEmpty#176, sum#177]
Keys [4]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174]
Functions [2]: [sum(sales#68), sum(number_sales#69)]
Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281]
Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130]
Aggregate Attributes [2]: [sum(sales#68)#178, sum(number_sales#69)#179]
Results [3]: [channel#73, sum(sales#68)#178 AS sum_sales#129, sum(number_sales#69)#179 AS number_sales#130]
(187) HashAggregate [codegen id : 562]
(145) HashAggregate [codegen id : 562]
Input [3]: [channel#73, sum_sales#129, number_sales#130]
Keys [1]: [channel#73]
Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)]
Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284]
Results [4]: [channel#73, sum#285, isEmpty#286, sum#287]
Aggregate Attributes [3]: [sum#180, isEmpty#181, sum#182]
Results [4]: [channel#73, sum#183, isEmpty#184, sum#185]
(188) Exchange
Input [4]: [channel#73, sum#285, isEmpty#286, sum#287]
Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288]
(146) Exchange
Input [4]: [channel#73, sum#183, isEmpty#184, sum#185]
Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#186]
(189) HashAggregate [codegen id : 563]
Input [4]: [channel#73, sum#285, isEmpty#286, sum#287]
(147) HashAggregate [codegen id : 563]
Input [4]: [channel#73, sum#183, isEmpty#184, sum#185]
Keys [1]: [channel#73]
Functions [2]: [sum(sum_sales#129), sum(number_sales#130)]
Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290]
Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295]
Aggregate Attributes [2]: [sum(sum_sales#129)#187, sum(number_sales#130)#188]
Results [6]: [channel#73, null AS i_brand_id#189, null AS i_class_id#190, null AS i_category_id#191, sum(sum_sales#129)#187 AS sum(sum_sales)#192, sum(number_sales#130)#188 AS sum(number_sales)#193]
(190) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301]
(148) ReusedExchange [Reuses operator id: unknown]
Output [7]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196, sum#197, isEmpty#198, sum#199]
(191) HashAggregate [codegen id : 609]
Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301]
Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305]
Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306]
(192) Filter [codegen id : 609]
Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(193) Project [codegen id : 609]
Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69]
Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306]
(194) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312]
(195) HashAggregate [codegen id : 655]
Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312]
Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316]
Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317]
(196) Filter [codegen id : 655]
Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(197) Project [codegen id : 655]
Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94]
Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317]
(198) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323]
(199) HashAggregate [codegen id : 701]
Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323]
Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327]
Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328]
(200) Filter [codegen id : 701]
Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6))))
(201) Project [codegen id : 701]
Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117]
Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328]
(202) Union
(203) HashAggregate [codegen id : 702]
Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69]
Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298]
Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)]
Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331]
Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334]
(204) Exchange
Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334]
Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335]
(205) HashAggregate [codegen id : 703]
Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334]
Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298]
(149) HashAggregate [codegen id : 703]
Input [7]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196, sum#197, isEmpty#198, sum#199]
Keys [4]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196]
Functions [2]: [sum(sales#68), sum(number_sales#69)]
Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337]
Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130]
Aggregate Attributes [2]: [sum(sales#68)#200, sum(number_sales#69)#201]
Results [2]: [sum(sales#68)#200 AS sum_sales#129, sum(number_sales#69)#201 AS number_sales#130]
(206) HashAggregate [codegen id : 703]
(150) HashAggregate [codegen id : 703]
Input [2]: [sum_sales#129, number_sales#130]
Keys: []
Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)]
Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340]
Results [3]: [sum#341, isEmpty#342, sum#343]
Aggregate Attributes [3]: [sum#202, isEmpty#203, sum#204]
Results [3]: [sum#205, isEmpty#206, sum#207]
(207) Exchange
Input [3]: [sum#341, isEmpty#342, sum#343]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344]
(151) Exchange
Input [3]: [sum#205, isEmpty#206, sum#207]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208]
(208) HashAggregate [codegen id : 704]
Input [3]: [sum#341, isEmpty#342, sum#343]
(152) HashAggregate [codegen id : 704]
Input [3]: [sum#205, isEmpty#206, sum#207]
Keys: []
Functions [2]: [sum(sum_sales#129), sum(number_sales#130)]
Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346]
Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352]
Aggregate Attributes [2]: [sum(sum_sales#129)#209, sum(number_sales#130)#210]
Results [6]: [null AS channel#211, null AS i_brand_id#212, null AS i_class_id#213, null AS i_category_id#214, sum(sum_sales#129)#209 AS sum(sum_sales)#215, sum(number_sales#130)#210 AS sum(number_sales)#216]
(209) Union
(153) Union
(210) HashAggregate [codegen id : 705]
(154) HashAggregate [codegen id : 705]
Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
(211) Exchange
(155) Exchange
Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353]
Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#217]
(212) HashAggregate [codegen id : 706]
(156) HashAggregate [codegen id : 706]
Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
(213) TakeOrderedAndProject
(157) TakeOrderedAndProject
Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130]
===== Subqueries =====
Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72]
* HashAggregate (240)
+- Exchange (239)
+- * HashAggregate (238)
+- Union (237)
:- * Project (222)
: +- * BroadcastHashJoin Inner BuildRight (221)
: :- * ColumnarToRow (215)
: : +- Scan parquet default.store_sales (214)
: +- BroadcastExchange (220)
: +- * Project (219)
: +- * Filter (218)
: +- * ColumnarToRow (217)
: +- Scan parquet default.date_dim (216)
:- * Project (231)
: +- * BroadcastHashJoin Inner BuildRight (230)
: :- * ColumnarToRow (224)
: : +- Scan parquet default.catalog_sales (223)
: +- BroadcastExchange (229)
: +- * Project (228)
: +- * Filter (227)
: +- * ColumnarToRow (226)
: +- Scan parquet default.date_dim (225)
+- * Project (236)
+- * BroadcastHashJoin Inner BuildRight (235)
:- * ColumnarToRow (233)
: +- Scan parquet default.web_sales (232)
+- ReusedExchange (234)
* HashAggregate (184)
+- Exchange (183)
+- * HashAggregate (182)
+- Union (181)
:- * Project (166)
: +- * BroadcastHashJoin Inner BuildRight (165)
: :- * ColumnarToRow (159)
: : +- Scan parquet default.store_sales (158)
: +- BroadcastExchange (164)
: +- * Project (163)
: +- * Filter (162)
: +- * ColumnarToRow (161)
: +- Scan parquet default.date_dim (160)
:- * Project (175)
: +- * BroadcastHashJoin Inner BuildRight (174)
: :- * ColumnarToRow (168)
: : +- Scan parquet default.catalog_sales (167)
: +- BroadcastExchange (173)
: +- * Project (172)
: +- * Filter (171)
: +- * ColumnarToRow (170)
: +- Scan parquet default.date_dim (169)
+- * Project (180)
+- * BroadcastHashJoin Inner BuildRight (179)
:- * ColumnarToRow (177)
: +- Scan parquet default.web_sales (176)
+- ReusedExchange (178)
(214) Scan parquet default.store_sales
Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356]
(158) Scan parquet default.store_sales
Output [3]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)]
PartitionFilters: [isnotnull(ss_sold_date_sk#220), dynamicpruningexpression(ss_sold_date_sk#220 IN dynamicpruning#221)]
ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
(215) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356]
(159) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220]
(216) Scan parquet default.date_dim
Output [2]: [d_date_sk#358, d_year#359]
(160) Scan parquet default.date_dim
Output [2]: [d_date_sk#222, d_year#223]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
(217) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#358, d_year#359]
(161) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#222, d_year#223]
(218) Filter [codegen id : 1]
Input [2]: [d_date_sk#358, d_year#359]
Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358))
(162) Filter [codegen id : 1]
Input [2]: [d_date_sk#222, d_year#223]
Condition : (((isnotnull(d_year#223) AND (d_year#223 >= 1999)) AND (d_year#223 <= 2001)) AND isnotnull(d_date_sk#222))
(219) Project [codegen id : 1]
Output [1]: [d_date_sk#358]
Input [2]: [d_date_sk#358, d_year#359]
(163) Project [codegen id : 1]
Output [1]: [d_date_sk#222]
Input [2]: [d_date_sk#222, d_year#223]
(220) BroadcastExchange
Input [1]: [d_date_sk#358]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360]
(164) BroadcastExchange
Input [1]: [d_date_sk#222]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#224]
(221) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#356]
Right keys [1]: [d_date_sk#358]
(165) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#220]
Right keys [1]: [d_date_sk#222]
Join condition: None
(222) Project [codegen id : 2]
Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362]
Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358]
(166) Project [codegen id : 2]
Output [2]: [ss_quantity#218 AS quantity#225, ss_list_price#219 AS list_price#226]
Input [4]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220, d_date_sk#222]
(223) Scan parquet default.catalog_sales
Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365]
(167) Scan parquet default.catalog_sales
Output [3]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)]
PartitionFilters: [isnotnull(cs_sold_date_sk#229), dynamicpruningexpression(cs_sold_date_sk#229 IN dynamicpruning#230)]
ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
(224) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365]
(168) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229]
(225) Scan parquet default.date_dim
Output [2]: [d_date_sk#367, d_year#368]
(169) Scan parquet default.date_dim
Output [2]: [d_date_sk#231, d_year#232]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
(226) ColumnarToRow [codegen id : 3]
Input [2]: [d_date_sk#367, d_year#368]
(170) ColumnarToRow [codegen id : 3]
Input [2]: [d_date_sk#231, d_year#232]
(227) Filter [codegen id : 3]
Input [2]: [d_date_sk#367, d_year#368]
Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367))
(171) Filter [codegen id : 3]
Input [2]: [d_date_sk#231, d_year#232]
Condition : (((isnotnull(d_year#232) AND (d_year#232 >= 1998)) AND (d_year#232 <= 2000)) AND isnotnull(d_date_sk#231))
(228) Project [codegen id : 3]
Output [1]: [d_date_sk#367]
Input [2]: [d_date_sk#367, d_year#368]
(172) Project [codegen id : 3]
Output [1]: [d_date_sk#231]
Input [2]: [d_date_sk#231, d_year#232]
(229) BroadcastExchange
Input [1]: [d_date_sk#367]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369]
(173) BroadcastExchange
Input [1]: [d_date_sk#231]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#233]
(230) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#365]
Right keys [1]: [d_date_sk#367]
(174) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#229]
Right keys [1]: [d_date_sk#231]
Join condition: None
(231) Project [codegen id : 4]
Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371]
Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367]
(175) Project [codegen id : 4]
Output [2]: [cs_quantity#227 AS quantity#234, cs_list_price#228 AS list_price#235]
Input [4]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229, d_date_sk#231]
(232) Scan parquet default.web_sales
Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374]
(176) Scan parquet default.web_sales
Output [3]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)]
PartitionFilters: [isnotnull(ws_sold_date_sk#238), dynamicpruningexpression(ws_sold_date_sk#238 IN dynamicpruning#230)]
ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
(233) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374]
(177) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238]
(234) ReusedExchange [Reuses operator id: 229]
Output [1]: [d_date_sk#375]
(178) ReusedExchange [Reuses operator id: 173]
Output [1]: [d_date_sk#239]
(235) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#374]
Right keys [1]: [d_date_sk#375]
(179) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#238]
Right keys [1]: [d_date_sk#239]
Join condition: None
(236) Project [codegen id : 6]
Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377]
Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375]
(180) Project [codegen id : 6]
Output [2]: [ws_quantity#236 AS quantity#240, ws_list_price#237 AS list_price#241]
Input [4]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238, d_date_sk#239]
(237) Union
(181) Union
(238) HashAggregate [codegen id : 7]
Input [2]: [quantity#361, list_price#362]
(182) HashAggregate [codegen id : 7]
Input [2]: [quantity#225, list_price#226]
Keys: []
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#378, count#379]
Results [2]: [sum#380, count#381]
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#242, count#243]
Results [2]: [sum#244, count#245]
(239) Exchange
Input [2]: [sum#380, count#381]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382]
(183) Exchange
Input [2]: [sum#244, count#245]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#246]
(240) HashAggregate [codegen id : 8]
Input [2]: [sum#380, count#381]
(184) HashAggregate [codegen id : 8]
Input [2]: [sum#244, count#245]
Keys: []
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384]
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))#247]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))#247 AS average_sales#248]
Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357
ReusedExchange (241)
Subquery:2 Hosting operator id = 158 Hosting Expression = ss_sold_date_sk#220 IN dynamicpruning#221
ReusedExchange (185)
(241) ReusedExchange [Reuses operator id: 220]
Output [1]: [d_date_sk#358]
(185) ReusedExchange [Reuses operator id: 164]
Output [1]: [d_date_sk#222]
Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366
ReusedExchange (242)
Subquery:3 Hosting operator id = 167 Hosting Expression = cs_sold_date_sk#229 IN dynamicpruning#230
ReusedExchange (186)
(242) ReusedExchange [Reuses operator id: 229]
Output [1]: [d_date_sk#367]
(186) ReusedExchange [Reuses operator id: 173]
Output [1]: [d_date_sk#231]
Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366
Subquery:4 Hosting operator id = 176 Hosting Expression = ws_sold_date_sk#238 IN dynamicpruning#230
Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
ReusedExchange (243)
ReusedExchange (187)
(243) ReusedExchange [Reuses operator id: 72]
(187) ReusedExchange [Reuses operator id: 72]
Output [1]: [d_date_sk#49]
Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13
ReusedExchange (244)
ReusedExchange (188)
(244) ReusedExchange [Reuses operator id: 16]
(188) ReusedExchange [Reuses operator id: 16]
Output [1]: [d_date_sk#14]
Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13
@ -1405,28 +1093,4 @@ Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subque
Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5
Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]
Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72]

View file

@ -300,131 +300,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #28
WholeStageCodegen (279)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (186)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29
WholeStageCodegen (232)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30
WholeStageCodegen (278)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28
WholeStageCodegen (422)
HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id] #32
Exchange [channel,i_brand_id] #29
WholeStageCodegen (421)
HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #33
WholeStageCodegen (420)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (327)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29
WholeStageCodegen (373)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30
WholeStageCodegen (419)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28
WholeStageCodegen (563)
HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange [channel] #34
Exchange [channel] #30
WholeStageCodegen (562)
HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #35
WholeStageCodegen (561)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (468)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29
WholeStageCodegen (514)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30
WholeStageCodegen (560)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28
WholeStageCodegen (704)
HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange #36
Exchange #31
WholeStageCodegen (703)
HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #37
WholeStageCodegen (702)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (609)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29
WholeStageCodegen (655)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30
WholeStageCodegen (701)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #28

View file

@ -1,9 +1,9 @@
== Physical Plan ==
TakeOrderedAndProject (194)
+- * HashAggregate (193)
+- Exchange (192)
+- * HashAggregate (191)
+- Union (190)
TakeOrderedAndProject (138)
+- * HashAggregate (137)
+- Exchange (136)
+- * HashAggregate (135)
+- Union (134)
:- * HashAggregate (113)
: +- Exchange (112)
: +- * HashAggregate (111)
@ -117,82 +117,26 @@ TakeOrderedAndProject (194)
: : : +- ReusedExchange (97)
: : +- ReusedExchange (99)
: +- ReusedExchange (102)
:- * HashAggregate (132)
: +- Exchange (131)
: +- * HashAggregate (130)
: +- * HashAggregate (129)
: +- Exchange (128)
: +- * HashAggregate (127)
: +- Union (126)
: :- * Project (117)
: : +- * Filter (116)
: : +- * HashAggregate (115)
: : +- ReusedExchange (114)
: :- * Project (121)
: : +- * Filter (120)
: : +- * HashAggregate (119)
: : +- ReusedExchange (118)
: +- * Project (125)
: +- * Filter (124)
: +- * HashAggregate (123)
: +- ReusedExchange (122)
:- * HashAggregate (151)
: +- Exchange (150)
: +- * HashAggregate (149)
: +- * HashAggregate (148)
: +- Exchange (147)
: +- * HashAggregate (146)
: +- Union (145)
: :- * Project (136)
: : +- * Filter (135)
: : +- * HashAggregate (134)
: : +- ReusedExchange (133)
: :- * Project (140)
: : +- * Filter (139)
: : +- * HashAggregate (138)
: : +- ReusedExchange (137)
: +- * Project (144)
: +- * Filter (143)
: +- * HashAggregate (142)
: +- ReusedExchange (141)
:- * HashAggregate (170)
: +- Exchange (169)
: +- * HashAggregate (168)
: +- * HashAggregate (167)
: +- Exchange (166)
: +- * HashAggregate (165)
: +- Union (164)
: :- * Project (155)
: : +- * Filter (154)
: : +- * HashAggregate (153)
: : +- ReusedExchange (152)
: :- * Project (159)
: : +- * Filter (158)
: : +- * HashAggregate (157)
: : +- ReusedExchange (156)
: +- * Project (163)
: +- * Filter (162)
: +- * HashAggregate (161)
: +- ReusedExchange (160)
+- * HashAggregate (189)
+- Exchange (188)
+- * HashAggregate (187)
+- * HashAggregate (186)
+- Exchange (185)
+- * HashAggregate (184)
+- Union (183)
:- * Project (174)
: +- * Filter (173)
: +- * HashAggregate (172)
: +- ReusedExchange (171)
:- * Project (178)
: +- * Filter (177)
: +- * HashAggregate (176)
: +- ReusedExchange (175)
+- * Project (182)
+- * Filter (181)
+- * HashAggregate (180)
+- ReusedExchange (179)
:- * HashAggregate (118)
: +- Exchange (117)
: +- * HashAggregate (116)
: +- * HashAggregate (115)
: +- ReusedExchange (114)
:- * HashAggregate (123)
: +- Exchange (122)
: +- * HashAggregate (121)
: +- * HashAggregate (120)
: +- ReusedExchange (119)
:- * HashAggregate (128)
: +- Exchange (127)
: +- * HashAggregate (126)
: +- * HashAggregate (125)
: +- ReusedExchange (124)
+- * HashAggregate (133)
+- Exchange (132)
+- * HashAggregate (131)
+- * HashAggregate (130)
+- ReusedExchange (129)
(1) Scan parquet default.store_sales
@ -721,581 +665,325 @@ Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121]
Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123]
(114) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129]
Output [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, sum#129]
(115) HashAggregate [codegen id : 106]
Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129]
Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133]
Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134]
(116) Filter [codegen id : 106]
Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(117) Project [codegen id : 106]
Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64]
Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134]
(118) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140]
(119) HashAggregate [codegen id : 132]
Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140]
Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144]
Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145]
(120) Filter [codegen id : 132]
Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(121) Project [codegen id : 132]
Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88]
Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145]
(122) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151]
(123) HashAggregate [codegen id : 158]
Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151]
Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155]
Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156]
(124) Filter [codegen id : 158]
Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(125) Project [codegen id : 158]
Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110]
Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156]
(126) Union
(127) HashAggregate [codegen id : 159]
Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64]
Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126]
Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)]
Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159]
Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162]
(128) Exchange
Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162]
Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163]
(129) HashAggregate [codegen id : 160]
Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162]
(115) HashAggregate [codegen id : 160]
Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, sum#129]
Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126]
Functions [2]: [sum(sales#63), sum(number_sales#64)]
Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165]
Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123]
Aggregate Attributes [2]: [sum(sales#63)#130, sum(number_sales#64)#131]
Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#130 AS sum_sales#122, sum(number_sales#64)#131 AS number_sales#123]
(130) HashAggregate [codegen id : 160]
(116) HashAggregate [codegen id : 160]
Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123]
Keys [3]: [channel#68, i_brand_id#124, i_class_id#125]
Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)]
Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168]
Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171]
Aggregate Attributes [3]: [sum#132, isEmpty#133, sum#134]
Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137]
(131) Exchange
Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171]
Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172]
(117) Exchange
Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137]
Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#138]
(132) HashAggregate [codegen id : 161]
Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171]
(118) HashAggregate [codegen id : 161]
Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137]
Keys [3]: [channel#68, i_brand_id#124, i_class_id#125]
Functions [2]: [sum(sum_sales#122), sum(number_sales#123)]
Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174]
Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177]
Aggregate Attributes [2]: [sum(sum_sales#122)#139, sum(number_sales#123)#140]
Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#141, sum(sum_sales#122)#139 AS sum(sum_sales)#142, sum(number_sales#123)#140 AS sum(number_sales)#143]
(133) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183]
(119) ReusedExchange [Reuses operator id: unknown]
Output [7]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146, sum#147, isEmpty#148, sum#149]
(134) HashAggregate [codegen id : 187]
Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183]
Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187]
Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188]
(135) Filter [codegen id : 187]
Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(136) Project [codegen id : 187]
Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64]
Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188]
(137) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194]
(138) HashAggregate [codegen id : 213]
Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194]
Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198]
Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199]
(139) Filter [codegen id : 213]
Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(140) Project [codegen id : 213]
Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88]
Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199]
(141) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205]
(142) HashAggregate [codegen id : 239]
Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205]
Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209]
Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210]
(143) Filter [codegen id : 239]
Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(144) Project [codegen id : 239]
Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110]
Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210]
(145) Union
(146) HashAggregate [codegen id : 240]
Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64]
Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180]
Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)]
Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213]
Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216]
(147) Exchange
Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216]
Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217]
(148) HashAggregate [codegen id : 241]
Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216]
Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180]
(120) HashAggregate [codegen id : 241]
Input [7]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146, sum#147, isEmpty#148, sum#149]
Keys [4]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146]
Functions [2]: [sum(sales#63), sum(number_sales#64)]
Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219]
Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123]
Aggregate Attributes [2]: [sum(sales#63)#150, sum(number_sales#64)#151]
Results [4]: [channel#68, i_brand_id#144, sum(sales#63)#150 AS sum_sales#122, sum(number_sales#64)#151 AS number_sales#123]
(149) HashAggregate [codegen id : 241]
Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123]
Keys [2]: [channel#68, i_brand_id#178]
(121) HashAggregate [codegen id : 241]
Input [4]: [channel#68, i_brand_id#144, sum_sales#122, number_sales#123]
Keys [2]: [channel#68, i_brand_id#144]
Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)]
Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222]
Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225]
Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154]
Results [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157]
(150) Exchange
Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225]
Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226]
(122) Exchange
Input [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157]
Arguments: hashpartitioning(channel#68, i_brand_id#144, 5), ENSURE_REQUIREMENTS, [id=#158]
(151) HashAggregate [codegen id : 242]
Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225]
Keys [2]: [channel#68, i_brand_id#178]
(123) HashAggregate [codegen id : 242]
Input [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157]
Keys [2]: [channel#68, i_brand_id#144]
Functions [2]: [sum(sum_sales#122), sum(number_sales#123)]
Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228]
Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232]
Aggregate Attributes [2]: [sum(sum_sales#122)#159, sum(number_sales#123)#160]
Results [6]: [channel#68, i_brand_id#144, null AS i_class_id#161, null AS i_category_id#162, sum(sum_sales#122)#159 AS sum(sum_sales)#163, sum(number_sales#123)#160 AS sum(number_sales)#164]
(152) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238]
(124) ReusedExchange [Reuses operator id: unknown]
Output [7]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167, sum#168, isEmpty#169, sum#170]
(153) HashAggregate [codegen id : 268]
Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238]
Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242]
Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243]
(154) Filter [codegen id : 268]
Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(155) Project [codegen id : 268]
Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64]
Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243]
(156) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249]
(157) HashAggregate [codegen id : 294]
Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249]
Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253]
Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254]
(158) Filter [codegen id : 294]
Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(159) Project [codegen id : 294]
Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88]
Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254]
(160) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260]
(161) HashAggregate [codegen id : 320]
Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260]
Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264]
Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265]
(162) Filter [codegen id : 320]
Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(163) Project [codegen id : 320]
Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110]
Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265]
(164) Union
(165) HashAggregate [codegen id : 321]
Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64]
Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235]
Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)]
Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268]
Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271]
(166) Exchange
Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271]
Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272]
(167) HashAggregate [codegen id : 322]
Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271]
Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235]
(125) HashAggregate [codegen id : 322]
Input [7]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167, sum#168, isEmpty#169, sum#170]
Keys [4]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167]
Functions [2]: [sum(sales#63), sum(number_sales#64)]
Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274]
Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123]
Aggregate Attributes [2]: [sum(sales#63)#171, sum(number_sales#64)#172]
Results [3]: [channel#68, sum(sales#63)#171 AS sum_sales#122, sum(number_sales#64)#172 AS number_sales#123]
(168) HashAggregate [codegen id : 322]
(126) HashAggregate [codegen id : 322]
Input [3]: [channel#68, sum_sales#122, number_sales#123]
Keys [1]: [channel#68]
Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)]
Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277]
Results [4]: [channel#68, sum#278, isEmpty#279, sum#280]
Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175]
Results [4]: [channel#68, sum#176, isEmpty#177, sum#178]
(169) Exchange
Input [4]: [channel#68, sum#278, isEmpty#279, sum#280]
Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281]
(127) Exchange
Input [4]: [channel#68, sum#176, isEmpty#177, sum#178]
Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#179]
(170) HashAggregate [codegen id : 323]
Input [4]: [channel#68, sum#278, isEmpty#279, sum#280]
(128) HashAggregate [codegen id : 323]
Input [4]: [channel#68, sum#176, isEmpty#177, sum#178]
Keys [1]: [channel#68]
Functions [2]: [sum(sum_sales#122), sum(number_sales#123)]
Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283]
Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288]
Aggregate Attributes [2]: [sum(sum_sales#122)#180, sum(number_sales#123)#181]
Results [6]: [channel#68, null AS i_brand_id#182, null AS i_class_id#183, null AS i_category_id#184, sum(sum_sales#122)#180 AS sum(sum_sales)#185, sum(number_sales#123)#181 AS sum(number_sales)#186]
(171) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294]
(129) ReusedExchange [Reuses operator id: unknown]
Output [7]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189, sum#190, isEmpty#191, sum#192]
(172) HashAggregate [codegen id : 349]
Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294]
Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298]
Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299]
(173) Filter [codegen id : 349]
Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(174) Project [codegen id : 349]
Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64]
Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299]
(175) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305]
(176) HashAggregate [codegen id : 375]
Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305]
Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309]
Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310]
(177) Filter [codegen id : 375]
Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(178) Project [codegen id : 375]
Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88]
Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310]
(179) ReusedExchange [Reuses operator id: unknown]
Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316]
(180) HashAggregate [codegen id : 401]
Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316]
Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313]
Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)]
Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320]
Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321]
(181) Filter [codegen id : 401]
Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321]
Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6))))
(182) Project [codegen id : 401]
Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110]
Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321]
(183) Union
(184) HashAggregate [codegen id : 402]
Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64]
Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291]
Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)]
Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324]
Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327]
(185) Exchange
Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327]
Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328]
(186) HashAggregate [codegen id : 403]
Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327]
Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291]
(130) HashAggregate [codegen id : 403]
Input [7]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189, sum#190, isEmpty#191, sum#192]
Keys [4]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189]
Functions [2]: [sum(sales#63), sum(number_sales#64)]
Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330]
Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123]
Aggregate Attributes [2]: [sum(sales#63)#193, sum(number_sales#64)#194]
Results [2]: [sum(sales#63)#193 AS sum_sales#122, sum(number_sales#64)#194 AS number_sales#123]
(187) HashAggregate [codegen id : 403]
(131) HashAggregate [codegen id : 403]
Input [2]: [sum_sales#122, number_sales#123]
Keys: []
Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)]
Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333]
Results [3]: [sum#334, isEmpty#335, sum#336]
Aggregate Attributes [3]: [sum#195, isEmpty#196, sum#197]
Results [3]: [sum#198, isEmpty#199, sum#200]
(188) Exchange
Input [3]: [sum#334, isEmpty#335, sum#336]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337]
(132) Exchange
Input [3]: [sum#198, isEmpty#199, sum#200]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#201]
(189) HashAggregate [codegen id : 404]
Input [3]: [sum#334, isEmpty#335, sum#336]
(133) HashAggregate [codegen id : 404]
Input [3]: [sum#198, isEmpty#199, sum#200]
Keys: []
Functions [2]: [sum(sum_sales#122), sum(number_sales#123)]
Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339]
Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345]
Aggregate Attributes [2]: [sum(sum_sales#122)#202, sum(number_sales#123)#203]
Results [6]: [null AS channel#204, null AS i_brand_id#205, null AS i_class_id#206, null AS i_category_id#207, sum(sum_sales#122)#202 AS sum(sum_sales)#208, sum(number_sales#123)#203 AS sum(number_sales)#209]
(190) Union
(134) Union
(191) HashAggregate [codegen id : 405]
(135) HashAggregate [codegen id : 405]
Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
(192) Exchange
(136) Exchange
Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346]
Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#210]
(193) HashAggregate [codegen id : 406]
(137) HashAggregate [codegen id : 406]
Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
Functions: []
Aggregate Attributes: []
Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
(194) TakeOrderedAndProject
(138) TakeOrderedAndProject
Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123]
===== Subqueries =====
Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67]
* HashAggregate (221)
+- Exchange (220)
+- * HashAggregate (219)
+- Union (218)
:- * Project (203)
: +- * BroadcastHashJoin Inner BuildRight (202)
: :- * ColumnarToRow (196)
: : +- Scan parquet default.store_sales (195)
: +- BroadcastExchange (201)
: +- * Project (200)
: +- * Filter (199)
: +- * ColumnarToRow (198)
: +- Scan parquet default.date_dim (197)
:- * Project (212)
: +- * BroadcastHashJoin Inner BuildRight (211)
: :- * ColumnarToRow (205)
: : +- Scan parquet default.catalog_sales (204)
: +- BroadcastExchange (210)
: +- * Project (209)
: +- * Filter (208)
: +- * ColumnarToRow (207)
: +- Scan parquet default.date_dim (206)
+- * Project (217)
+- * BroadcastHashJoin Inner BuildRight (216)
:- * ColumnarToRow (214)
: +- Scan parquet default.web_sales (213)
+- ReusedExchange (215)
* HashAggregate (165)
+- Exchange (164)
+- * HashAggregate (163)
+- Union (162)
:- * Project (147)
: +- * BroadcastHashJoin Inner BuildRight (146)
: :- * ColumnarToRow (140)
: : +- Scan parquet default.store_sales (139)
: +- BroadcastExchange (145)
: +- * Project (144)
: +- * Filter (143)
: +- * ColumnarToRow (142)
: +- Scan parquet default.date_dim (141)
:- * Project (156)
: +- * BroadcastHashJoin Inner BuildRight (155)
: :- * ColumnarToRow (149)
: : +- Scan parquet default.catalog_sales (148)
: +- BroadcastExchange (154)
: +- * Project (153)
: +- * Filter (152)
: +- * ColumnarToRow (151)
: +- Scan parquet default.date_dim (150)
+- * Project (161)
+- * BroadcastHashJoin Inner BuildRight (160)
:- * ColumnarToRow (158)
: +- Scan parquet default.web_sales (157)
+- ReusedExchange (159)
(195) Scan parquet default.store_sales
Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349]
(139) Scan parquet default.store_sales
Output [3]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)]
PartitionFilters: [isnotnull(ss_sold_date_sk#213), dynamicpruningexpression(ss_sold_date_sk#213 IN dynamicpruning#214)]
ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
(196) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349]
(140) ColumnarToRow [codegen id : 2]
Input [3]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213]
(197) Scan parquet default.date_dim
Output [2]: [d_date_sk#351, d_year#352]
(141) Scan parquet default.date_dim
Output [2]: [d_date_sk#215, d_year#216]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
(198) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#351, d_year#352]
(142) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#215, d_year#216]
(199) Filter [codegen id : 1]
Input [2]: [d_date_sk#351, d_year#352]
Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351))
(143) Filter [codegen id : 1]
Input [2]: [d_date_sk#215, d_year#216]
Condition : (((isnotnull(d_year#216) AND (d_year#216 >= 1999)) AND (d_year#216 <= 2001)) AND isnotnull(d_date_sk#215))
(200) Project [codegen id : 1]
Output [1]: [d_date_sk#351]
Input [2]: [d_date_sk#351, d_year#352]
(144) Project [codegen id : 1]
Output [1]: [d_date_sk#215]
Input [2]: [d_date_sk#215, d_year#216]
(201) BroadcastExchange
Input [1]: [d_date_sk#351]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353]
(145) BroadcastExchange
Input [1]: [d_date_sk#215]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#217]
(202) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#349]
Right keys [1]: [d_date_sk#351]
(146) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [ss_sold_date_sk#213]
Right keys [1]: [d_date_sk#215]
Join condition: None
(203) Project [codegen id : 2]
Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355]
Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351]
(147) Project [codegen id : 2]
Output [2]: [ss_quantity#211 AS quantity#218, ss_list_price#212 AS list_price#219]
Input [4]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213, d_date_sk#215]
(204) Scan parquet default.catalog_sales
Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358]
(148) Scan parquet default.catalog_sales
Output [3]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)]
PartitionFilters: [isnotnull(cs_sold_date_sk#222), dynamicpruningexpression(cs_sold_date_sk#222 IN dynamicpruning#223)]
ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
(205) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358]
(149) ColumnarToRow [codegen id : 4]
Input [3]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222]
(206) Scan parquet default.date_dim
Output [2]: [d_date_sk#360, d_year#361]
(150) Scan parquet default.date_dim
Output [2]: [d_date_sk#224, d_year#225]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
(207) ColumnarToRow [codegen id : 3]
Input [2]: [d_date_sk#360, d_year#361]
(151) ColumnarToRow [codegen id : 3]
Input [2]: [d_date_sk#224, d_year#225]
(208) Filter [codegen id : 3]
Input [2]: [d_date_sk#360, d_year#361]
Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360))
(152) Filter [codegen id : 3]
Input [2]: [d_date_sk#224, d_year#225]
Condition : (((isnotnull(d_year#225) AND (d_year#225 >= 1998)) AND (d_year#225 <= 2000)) AND isnotnull(d_date_sk#224))
(209) Project [codegen id : 3]
Output [1]: [d_date_sk#360]
Input [2]: [d_date_sk#360, d_year#361]
(153) Project [codegen id : 3]
Output [1]: [d_date_sk#224]
Input [2]: [d_date_sk#224, d_year#225]
(210) BroadcastExchange
Input [1]: [d_date_sk#360]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362]
(154) BroadcastExchange
Input [1]: [d_date_sk#224]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#226]
(211) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#358]
Right keys [1]: [d_date_sk#360]
(155) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [cs_sold_date_sk#222]
Right keys [1]: [d_date_sk#224]
Join condition: None
(212) Project [codegen id : 4]
Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364]
Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360]
(156) Project [codegen id : 4]
Output [2]: [cs_quantity#220 AS quantity#227, cs_list_price#221 AS list_price#228]
Input [4]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222, d_date_sk#224]
(213) Scan parquet default.web_sales
Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367]
(157) Scan parquet default.web_sales
Output [3]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)]
PartitionFilters: [isnotnull(ws_sold_date_sk#231), dynamicpruningexpression(ws_sold_date_sk#231 IN dynamicpruning#223)]
ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
(214) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367]
(158) ColumnarToRow [codegen id : 6]
Input [3]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231]
(215) ReusedExchange [Reuses operator id: 210]
Output [1]: [d_date_sk#368]
(159) ReusedExchange [Reuses operator id: 154]
Output [1]: [d_date_sk#232]
(216) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#367]
Right keys [1]: [d_date_sk#368]
(160) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ws_sold_date_sk#231]
Right keys [1]: [d_date_sk#232]
Join condition: None
(217) Project [codegen id : 6]
Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370]
Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368]
(161) Project [codegen id : 6]
Output [2]: [ws_quantity#229 AS quantity#233, ws_list_price#230 AS list_price#234]
Input [4]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231, d_date_sk#232]
(218) Union
(162) Union
(219) HashAggregate [codegen id : 7]
Input [2]: [quantity#354, list_price#355]
(163) HashAggregate [codegen id : 7]
Input [2]: [quantity#218, list_price#219]
Keys: []
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#371, count#372]
Results [2]: [sum#373, count#374]
Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [2]: [sum#235, count#236]
Results [2]: [sum#237, count#238]
(220) Exchange
Input [2]: [sum#373, count#374]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375]
(164) Exchange
Input [2]: [sum#237, count#238]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#239]
(221) HashAggregate [codegen id : 8]
Input [2]: [sum#373, count#374]
(165) HashAggregate [codegen id : 8]
Input [2]: [sum#237, count#238]
Keys: []
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377]
Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))]
Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))#240]
Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))#240 AS average_sales#241]
Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350
ReusedExchange (222)
Subquery:2 Hosting operator id = 139 Hosting Expression = ss_sold_date_sk#213 IN dynamicpruning#214
ReusedExchange (166)
(222) ReusedExchange [Reuses operator id: 201]
Output [1]: [d_date_sk#351]
(166) ReusedExchange [Reuses operator id: 145]
Output [1]: [d_date_sk#215]
Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359
ReusedExchange (223)
Subquery:3 Hosting operator id = 148 Hosting Expression = cs_sold_date_sk#222 IN dynamicpruning#223
ReusedExchange (167)
(223) ReusedExchange [Reuses operator id: 210]
Output [1]: [d_date_sk#360]
(167) ReusedExchange [Reuses operator id: 154]
Output [1]: [d_date_sk#224]
Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359
Subquery:4 Hosting operator id = 157 Hosting Expression = ws_sold_date_sk#231 IN dynamicpruning#223
Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
ReusedExchange (224)
ReusedExchange (168)
(224) ReusedExchange [Reuses operator id: 70]
(168) ReusedExchange [Reuses operator id: 70]
Output [1]: [d_date_sk#50]
Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
ReusedExchange (225)
ReusedExchange (169)
(225) ReusedExchange [Reuses operator id: 26]
(169) ReusedExchange [Reuses operator id: 26]
Output [1]: [d_date_sk#29]
Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12
@ -1310,28 +998,4 @@ Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subque
Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5
Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]
Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67]

View file

@ -243,131 +243,31 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #21
WholeStageCodegen (159)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (106)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22
WholeStageCodegen (132)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23
WholeStageCodegen (158)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21
WholeStageCodegen (242)
HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id] #25
Exchange [channel,i_brand_id] #22
WholeStageCodegen (241)
HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #26
WholeStageCodegen (240)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (187)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22
WholeStageCodegen (213)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23
WholeStageCodegen (239)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21
WholeStageCodegen (323)
HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange [channel] #27
Exchange [channel] #23
WholeStageCodegen (322)
HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #28
WholeStageCodegen (321)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (268)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22
WholeStageCodegen (294)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23
WholeStageCodegen (320)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21
WholeStageCodegen (404)
HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
InputAdapter
Exchange #29
Exchange #24
WholeStageCodegen (403)
HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
InputAdapter
Exchange [channel,i_brand_id,i_class_id,i_category_id] #30
WholeStageCodegen (402)
HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
InputAdapter
Union
WholeStageCodegen (349)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #22
WholeStageCodegen (375)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23
WholeStageCodegen (401)
Project [i_brand_id,i_class_id,i_category_id,sales,number_sales]
Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))]
ReusedSubquery [average_sales] #3
HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count]
InputAdapter
ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #24
ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #21