diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala index fa41e865444da..3cbe1654ea2cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala @@ -16,8 +16,8 @@ */ package org.apache.spark.sql.execution -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, NamedExpression, SortOrder} -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeReference, Expression, NamedExpression, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning /** * A trait that provides functionality to handle aliases in the `outputExpressions`. @@ -25,19 +25,15 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partition trait AliasAwareOutputExpression extends UnaryExecNode { protected def outputExpressions: Seq[NamedExpression] - protected def hasAlias: Boolean = outputExpressions.collectFirst { case _: Alias => }.isDefined + private lazy val aliasMap = AttributeMap(outputExpressions.collect { + case a @ Alias(child: AttributeReference, _) => (child, a.toAttribute) + }) - protected def replaceAliases(exprs: Seq[Expression]): Seq[Expression] = { - exprs.map { - case a: AttributeReference => replaceAlias(a).getOrElse(a) - case other => other - } - } + protected def hasAlias: Boolean = aliasMap.nonEmpty - protected def replaceAlias(attr: AttributeReference): Option[Attribute] = { - outputExpressions.collectFirst { - case a @ Alias(child: AttributeReference, _) if child.semanticEquals(attr) => - a.toAttribute + protected def normalizeExpression(exp: Expression): Expression = { + exp.transform { + case attr: AttributeReference => aliasMap.getOrElse(attr, attr) } } } @@ -50,7 +46,8 @@ trait AliasAwareOutputPartitioning extends AliasAwareOutputExpression { final override def outputPartitioning: Partitioning = { if (hasAlias) { child.outputPartitioning match { - case h: HashPartitioning => h.copy(expressions = replaceAliases(h.expressions)) + case e: Expression => + normalizeExpression(e).asInstanceOf[Partitioning] case other => other } } else { @@ -68,12 +65,7 @@ trait AliasAwareOutputOrdering extends AliasAwareOutputExpression { final override def outputOrdering: Seq[SortOrder] = { if (hasAlias) { - orderingExpressions.map { s => - s.child match { - case a: AttributeReference => s.copy(child = replaceAlias(a).getOrElse(a)) - case _ => s - } - } + orderingExpressions.map(normalizeExpression(_).asInstanceOf[SortOrder]) } else { orderingExpressions } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index fe5966bb4dfb3..61e5ae0121819 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -1,46 +1,45 @@ == Physical Plan == -* Sort (42) -+- Exchange (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (27) - : +- Exchange (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * HashAggregate (18) - : : +- Exchange (17) - : : +- * HashAggregate (16) - : : +- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.catalog_sales (5) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.date_dim (10) - : +- BroadcastExchange (23) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.date_dim (19) - +- * Sort (38) - +- Exchange (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * HashAggregate (29) - : +- ReusedExchange (28) - +- BroadcastExchange (34) - +- * Project (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.date_dim (30) +* Sort (41) ++- Exchange (40) + +- * Project (39) + +- * SortMergeJoin Inner (38) + :- * Sort (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * HashAggregate (18) + : : +- Exchange (17) + : : +- * HashAggregate (16) + : : +- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- Union (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.web_sales (1) + : : : +- * Project (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.catalog_sales (5) + : : +- BroadcastExchange (13) + : : +- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet default.date_dim (10) + : +- BroadcastExchange (23) + : +- * Project (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.date_dim (19) + +- * Sort (37) + +- Exchange (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * HashAggregate (28) + : +- ReusedExchange (27) + +- BroadcastExchange (33) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.date_dim (29) (1) Scan parquet default.web_sales @@ -157,77 +156,73 @@ Join condition: None Output [8]: [d_week_seq#10 AS d_week_seq1#45, sun_sales#35 AS sun_sales1#46, mon_sales#36 AS mon_sales1#47, tue_sales#37 AS tue_sales1#48, wed_sales#38 AS wed_sales1#49, thu_sales#39 AS thu_sales1#50, fri_sales#40 AS fri_sales1#51, sat_sales#41 AS sat_sales1#52] Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#42] -(26) Exchange -Input [8]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52] -Arguments: hashpartitioning(d_week_seq1#45, 5), true, [id=#53] - -(27) Sort [codegen id : 7] +(26) Sort [codegen id : 6] Input [8]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52] Arguments: [d_week_seq1#45 ASC NULLS FIRST], false, 0 -(28) ReusedExchange [Reuses operator id: 17] -Output [8]: [d_week_seq#10, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +(27) ReusedExchange [Reuses operator id: 17] +Output [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] -(29) HashAggregate [codegen id : 13] -Input [8]: [d_week_seq#10, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +(28) HashAggregate [codegen id : 12] +Input [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))#67] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END))#61,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END))#62,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END))#63,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#64,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END))#65,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END))#66,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))#67,17,2) AS sat_sales#41] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))#66] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] -(30) Scan parquet default.date_dim -Output [2]: [d_week_seq#68, d_year#69] +(29) Scan parquet default.date_dim +Output [2]: [d_week_seq#67, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 12] -Input [2]: [d_week_seq#68, d_year#69] +(30) ColumnarToRow [codegen id : 11] +Input [2]: [d_week_seq#67, d_year#68] -(32) Filter [codegen id : 12] -Input [2]: [d_week_seq#68, d_year#69] -Condition : ((isnotnull(d_year#69) AND (d_year#69 = 2002)) AND isnotnull(d_week_seq#68)) +(31) Filter [codegen id : 11] +Input [2]: [d_week_seq#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) -(33) Project [codegen id : 12] -Output [1]: [d_week_seq#68] -Input [2]: [d_week_seq#68, d_year#69] +(32) Project [codegen id : 11] +Output [1]: [d_week_seq#67] +Input [2]: [d_week_seq#67, d_year#68] -(34) BroadcastExchange -Input [1]: [d_week_seq#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] +(33) BroadcastExchange +Input [1]: [d_week_seq#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] -(35) BroadcastHashJoin [codegen id : 13] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#68] +Right keys [1]: [d_week_seq#67] Join condition: None -(36) Project [codegen id : 13] -Output [8]: [d_week_seq#10 AS d_week_seq2#71, sun_sales#35 AS sun_sales2#72, mon_sales#36 AS mon_sales2#73, tue_sales#37 AS tue_sales2#74, wed_sales#38 AS wed_sales2#75, thu_sales#39 AS thu_sales2#76, fri_sales#40 AS fri_sales2#77, sat_sales#41 AS sat_sales2#78] -Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#68] +(35) Project [codegen id : 12] +Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] +Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#67] -(37) Exchange -Input [8]: [d_week_seq2#71, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78] -Arguments: hashpartitioning((d_week_seq2#71 - 53), 5), true, [id=#79] +(36) Exchange +Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +Arguments: hashpartitioning((d_week_seq2#70 - 53), 5), true, [id=#78] -(38) Sort [codegen id : 14] -Input [8]: [d_week_seq2#71, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78] -Arguments: [(d_week_seq2#71 - 53) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 13] +Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +Arguments: [(d_week_seq2#70 - 53) ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 15] +(38) SortMergeJoin [codegen id : 14] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#71 - 53)] +Right keys [1]: [(d_week_seq2#70 - 53)] Join condition: None -(40) Project [codegen id : 15] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#72)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#80, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#73)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#81, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#74)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#82, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#75)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#83, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#76)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#84, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#77)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#85, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#78)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#86] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#71, sun_sales2#72, mon_sales2#73, tue_sales2#74, wed_sales2#75, thu_sales2#76, fri_sales2#77, sat_sales2#78] +(39) Project [codegen id : 14] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -(41) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#80, round((mon_sales1 / mon_sales2), 2)#81, round((tue_sales1 / tue_sales2), 2)#82, round((wed_sales1 / wed_sales2), 2)#83, round((thu_sales1 / thu_sales2), 2)#84, round((fri_sales1 / fri_sales2), 2)#85, round((sat_sales1 / sat_sales2), 2)#86] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), true, [id=#87] +(40) Exchange +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), true, [id=#86] -(42) Sort [codegen id : 16] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#80, round((mon_sales1 / mon_sales2), 2)#81, round((tue_sales1 / tue_sales2), 2)#82, round((wed_sales1 / wed_sales2), 2)#83, round((thu_sales1 / thu_sales2), 2)#84, round((fri_sales1 / fri_sales2), 2)#85, round((sat_sales1 / sat_sales2), 2)#86] +(41) Sort [codegen id : 15] +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] Arguments: [d_week_seq1#45 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index 3df7e4c8e6f3f..3389774c46469 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -1,68 +1,65 @@ -WholeStageCodegen (16) +WholeStageCodegen (15) Sort [d_week_seq1] InputAdapter Exchange [d_week_seq1] #1 - WholeStageCodegen (15) + WholeStageCodegen (14) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] SortMergeJoin [d_week_seq1,d_week_seq2] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (6) Sort [d_week_seq1] - InputAdapter - Exchange [d_week_seq1] #2 - WholeStageCodegen (6) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - Exchange [d_week_seq] #3 - WholeStageCodegen (4) - HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ws_sold_date_sk,ws_ext_sales_price] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] - WholeStageCodegen (2) - Project [cs_sold_date_sk,cs_ext_sales_price] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [d_date_sk,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [d_week_seq] #2 + WholeStageCodegen (4) + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (1) + Project [ws_sold_date_sk,ws_ext_sales_price] + Filter [ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] + WholeStageCodegen (2) + Project [cs_sold_date_sk,cs_ext_sales_price] + Filter [cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (3) + Filter [d_date_sk,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + Project [d_week_seq] + Filter [d_year,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_week_seq,d_year] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (13) Sort [d_week_seq2] InputAdapter - Exchange [d_week_seq2] #6 - WholeStageCodegen (13) + Exchange [d_week_seq2] #5 + WholeStageCodegen (12) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #3 + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (12) + BroadcastExchange #6 + WholeStageCodegen (11) Project [d_week_seq] Filter [d_year,d_week_seq] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index c5988072f758d..bda9824b71b5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -1,96 +1,104 @@ == Physical Plan == -CollectLimit (92) -+- * HashAggregate (91) - +- Exchange (90) - +- * HashAggregate (89) - +- Union (88) - :- * Project (60) - : +- * BroadcastHashJoin Inner BuildRight (59) - : :- * Project (53) - : : +- SortMergeJoin LeftSemi (52) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- SortMergeJoin LeftSemi (31) +CollectLimit (100) ++- * HashAggregate (99) + +- Exchange (98) + +- * HashAggregate (97) + +- Union (96) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * Project (52) + : : +- SortMergeJoin LeftSemi (51) + : : :- * Sort (33) + : : : +- Exchange (32) + : : : +- * Project (31) + : : : +- SortMergeJoin LeftSemi (30) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.catalog_sales (1) - : : : +- * Sort (30) - : : : +- Exchange (29) - : : : +- * Project (28) - : : : +- * Filter (27) - : : : +- * HashAggregate (26) - : : : +- * HashAggregate (25) - : : : +- * Project (24) - : : : +- * SortMergeJoin Inner (23) - : : : :- * Sort (17) - : : : : +- Exchange (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.store_sales (6) - : : : : +- BroadcastExchange (13) - : : : : +- * Project (12) - : : : : +- * Filter (11) - : : : : +- * ColumnarToRow (10) - : : : : +- Scan parquet default.date_dim (9) - : : : +- * Sort (22) - : : : +- Exchange (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.item (18) - : : +- * Sort (51) - : : +- * Project (50) - : : +- * Filter (49) - : : +- * HashAggregate (48) - : : +- * HashAggregate (47) - : : +- * Project (46) - : : +- * SortMergeJoin Inner (45) - : : :- * Sort (39) - : : : +- Exchange (38) - : : : +- * Filter (37) - : : : +- * ColumnarToRow (36) - : : : +- Scan parquet default.store_sales (35) - : : +- * Sort (44) - : : +- Exchange (43) - : : +- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.customer (40) - : +- BroadcastExchange (58) - : +- * Project (57) - : +- * Filter (56) - : +- * ColumnarToRow (55) - : +- Scan parquet default.date_dim (54) - +- * Project (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Project (84) - : +- SortMergeJoin LeftSemi (83) - : :- * Sort (71) - : : +- Exchange (70) - : : +- * Project (69) - : : +- SortMergeJoin LeftSemi (68) - : : :- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.web_sales (61) - : : +- * Sort (67) - : : +- ReusedExchange (66) - : +- * Sort (82) - : +- * Project (81) - : +- * Filter (80) - : +- * HashAggregate (79) - : +- * HashAggregate (78) - : +- * Project (77) - : +- * SortMergeJoin Inner (76) - : :- * Sort (73) - : : +- ReusedExchange (72) - : +- * Sort (75) - : +- ReusedExchange (74) - +- ReusedExchange (85) + : : : +- * Sort (29) + : : : +- * Project (28) + : : : +- * Filter (27) + : : : +- * HashAggregate (26) + : : : +- * HashAggregate (25) + : : : +- * Project (24) + : : : +- * SortMergeJoin Inner (23) + : : : :- * Sort (17) + : : : : +- Exchange (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * Filter (11) + : : : : +- * ColumnarToRow (10) + : : : : +- Scan parquet default.date_dim (9) + : : : +- * Sort (22) + : : : +- Exchange (21) + : : : +- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet default.item (18) + : : +- * Sort (50) + : : +- * Project (49) + : : +- * Filter (48) + : : +- * HashAggregate (47) + : : +- * HashAggregate (46) + : : +- * Project (45) + : : +- * SortMergeJoin Inner (44) + : : :- * Sort (38) + : : : +- Exchange (37) + : : : +- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet default.store_sales (34) + : : +- * Sort (43) + : : +- Exchange (42) + : : +- * Filter (41) + : : +- * ColumnarToRow (40) + : : +- Scan parquet default.customer (39) + : +- BroadcastExchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * ColumnarToRow (54) + : +- Scan parquet default.date_dim (53) + +- * Project (95) + +- * BroadcastHashJoin Inner BuildRight (94) + :- * Project (92) + : +- SortMergeJoin LeftSemi (91) + : :- * Sort (79) + : : +- Exchange (78) + : : +- * Project (77) + : : +- SortMergeJoin LeftSemi (76) + : : :- * Sort (64) + : : : +- Exchange (63) + : : : +- * Filter (62) + : : : +- * ColumnarToRow (61) + : : : +- Scan parquet default.web_sales (60) + : : +- * Sort (75) + : : +- * Project (74) + : : +- * Filter (73) + : : +- * HashAggregate (72) + : : +- * HashAggregate (71) + : : +- * Project (70) + : : +- * SortMergeJoin Inner (69) + : : :- * Sort (66) + : : : +- ReusedExchange (65) + : : +- * Sort (68) + : : +- ReusedExchange (67) + : +- * Sort (90) + : +- * Project (89) + : +- * Filter (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- * Project (85) + : +- * SortMergeJoin Inner (84) + : :- * Sort (81) + : : +- ReusedExchange (80) + : +- * Sort (83) + : +- ReusedExchange (82) + +- ReusedExchange (93) (1) Scan parquet default.catalog_sales @@ -221,435 +229,473 @@ Condition : (count(1)#22 > 4) Output [1]: [item_sk#21] Input [2]: [item_sk#21, count(1)#22] -(29) Exchange -Input [1]: [item_sk#21] -Arguments: hashpartitioning(item_sk#21, 5), true, [id=#23] - -(30) Sort [codegen id : 9] +(29) Sort [codegen id : 8] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin +(30) SortMergeJoin Left keys [1]: [cs_item_sk#3] Right keys [1]: [item_sk#21] Join condition: None -(32) Project [codegen id : 10] +(31) Project [codegen id : 9] Output [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -(33) Exchange +(32) Exchange Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#24] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#23] -(34) Sort [codegen id : 11] +(33) Sort [codegen id : 10] Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(34) Scan parquet default.store_sales +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 12] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(35) ColumnarToRow [codegen id : 11] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(37) Filter [codegen id : 12] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Condition : isnotnull(ss_customer_sk#25) +(36) Filter [codegen id : 11] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Condition : isnotnull(ss_customer_sk#24) -(38) Exchange -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: hashpartitioning(ss_customer_sk#25, 5), true, [id=#28] +(37) Exchange +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#27] -(39) Sort [codegen id : 13] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(38) Sort [codegen id : 12] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(40) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +(39) Scan parquet default.customer +Output [1]: [c_customer_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 14] -Input [1]: [c_customer_sk#29] +(40) ColumnarToRow [codegen id : 13] +Input [1]: [c_customer_sk#28] -(42) Filter [codegen id : 14] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +(41) Filter [codegen id : 13] +Input [1]: [c_customer_sk#28] +Condition : isnotnull(c_customer_sk#28) -(43) Exchange -Input [1]: [c_customer_sk#29] -Arguments: hashpartitioning(c_customer_sk#29, 5), true, [id=#30] +(42) Exchange +Input [1]: [c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#29] -(44) Sort [codegen id : 15] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(43) Sort [codegen id : 14] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 16] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(44) SortMergeJoin [codegen id : 15] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(46) Project [codegen id : 16] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(47) HashAggregate [codegen id : 16] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(48) HashAggregate [codegen id : 16] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#35] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#35 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] - -(49) Filter [codegen id : 16] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) - -(50) Project [codegen id : 16] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] - -(51) Sort [codegen id : 16] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 - -(52) SortMergeJoin +(45) Project [codegen id : 15] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(46) HashAggregate [codegen id : 15] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#30, isEmpty#31] +Results [3]: [c_customer_sk#28, sum#32, isEmpty#33] + +(47) HashAggregate [codegen id : 15] +Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] + +(48) Filter [codegen id : 15] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) + +(49) Project [codegen id : 15] +Output [1]: [c_customer_sk#28] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] + +(50) Sort [codegen id : 15] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#28] Join condition: None -(53) Project [codegen id : 18] +(52) Project [codegen id : 17] Output [3]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5] Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -(54) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#11, d_moy#39] +(53) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#11, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] -Input [3]: [d_date_sk#9, d_year#11, d_moy#39] +(54) ColumnarToRow [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#38] -(56) Filter [codegen id : 17] -Input [3]: [d_date_sk#9, d_year#11, d_moy#39] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#39)) AND (d_year#11 = 2000)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#9)) +(55) Filter [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#38] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#38)) AND (d_year#11 = 2000)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#9)) -(57) Project [codegen id : 17] +(56) Project [codegen id : 16] Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#11, d_moy#39] +Input [3]: [d_date_sk#9, d_year#11, d_moy#38] -(58) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] -(59) BroadcastHashJoin [codegen id : 18] +(58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#1] Right keys [1]: [d_date_sk#9] Join condition: None -(60) Project [codegen id : 18] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true) AS sales#41] +(59) Project [codegen id : 17] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true) AS sales#40] Input [4]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, d_date_sk#9] -(61) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] +(60) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 19] -Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] +(61) ColumnarToRow [codegen id : 18] +Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] + +(62) Filter [codegen id : 18] +Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +Condition : isnotnull(ws_sold_date_sk#41) + +(63) Exchange +Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +Arguments: hashpartitioning(ws_item_sk#42, 5), true, [id=#46] -(63) Filter [codegen id : 19] -Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Condition : isnotnull(ws_sold_date_sk#42) +(64) Sort [codegen id : 19] +Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +Arguments: [ws_item_sk#42 ASC NULLS FIRST], false, 0 -(64) Exchange -Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Arguments: hashpartitioning(ws_item_sk#43, 5), true, [id=#47] +(65) ReusedExchange [Reuses operator id: 16] +Output [2]: [ss_item_sk#8, d_date#10] + +(66) Sort [codegen id : 22] +Input [2]: [ss_item_sk#8, d_date#10] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 + +(67) ReusedExchange [Reuses operator id: 21] +Output [2]: [i_item_sk#14, i_item_desc#15] + +(68) Sort [codegen id : 24] +Input [2]: [i_item_sk#14, i_item_desc#15] +Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 + +(69) SortMergeJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#14] +Join condition: None + +(70) Project [codegen id : 25] +Output [3]: [d_date#10, i_item_sk#14, i_item_desc#15] +Input [4]: [ss_item_sk#8, d_date#10, i_item_sk#14, i_item_desc#15] + +(71) HashAggregate [codegen id : 25] +Input [3]: [d_date#10, i_item_sk#14, i_item_desc#15] +Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#48] +Results [4]: [substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10, count#49] + +(72) HashAggregate [codegen id : 25] +Input [4]: [substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10, count#49] +Keys [3]: [substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#50] +Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#50 AS count(1)#51] -(65) Sort [codegen id : 20] -Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 +(73) Filter [codegen id : 25] +Input [2]: [item_sk#21, count(1)#51] +Condition : (count(1)#51 > 4) -(66) ReusedExchange [Reuses operator id: 29] +(74) Project [codegen id : 25] Output [1]: [item_sk#21] +Input [2]: [item_sk#21, count(1)#51] -(67) Sort [codegen id : 27] +(75) Sort [codegen id : 25] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin -Left keys [1]: [ws_item_sk#43] +(76) SortMergeJoin +Left keys [1]: [ws_item_sk#42] Right keys [1]: [item_sk#21] Join condition: None -(69) Project [codegen id : 28] -Output [4]: [ws_sold_date_sk#42, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Input [5]: [ws_sold_date_sk#42, ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] +(77) Project [codegen id : 26] +Output [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -(70) Exchange -Input [4]: [ws_sold_date_sk#42, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), true, [id=#48] +(78) Exchange +Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +Arguments: hashpartitioning(ws_bill_customer_sk#43, 5), true, [id=#52] -(71) Sort [codegen id : 29] -Input [4]: [ws_sold_date_sk#42, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 27] +Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +Arguments: [ws_bill_customer_sk#43 ASC NULLS FIRST], false, 0 -(72) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(80) ReusedExchange [Reuses operator id: 37] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(73) Sort [codegen id : 31] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 29] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(74) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#29] +(82) ReusedExchange [Reuses operator id: 42] +Output [1]: [c_customer_sk#28] -(75) Sort [codegen id : 33] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(83) Sort [codegen id : 31] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin [codegen id : 34] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(84) SortMergeJoin [codegen id : 32] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(77) Project [codegen id : 34] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(78) HashAggregate [codegen id : 34] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [3]: [c_customer_sk#29, sum#51, isEmpty#52] - -(79) HashAggregate [codegen id : 34] -Input [3]: [c_customer_sk#29, sum#51, isEmpty#52] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#53] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#53 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#54] - -(80) Filter [codegen id : 34] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#54] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#54) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#54 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) - -(81) Project [codegen id : 34] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#54] - -(82) Sort [codegen id : 34] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 - -(83) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#29] +(85) Project [codegen id : 32] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(86) HashAggregate [codegen id : 32] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#53, isEmpty#54] +Results [3]: [c_customer_sk#28, sum#55, isEmpty#56] + +(87) HashAggregate [codegen id : 32] +Input [3]: [c_customer_sk#28, sum#55, isEmpty#56] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#57] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58] + +(88) Filter [codegen id : 32] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) + +(89) Project [codegen id : 32] +Output [1]: [c_customer_sk#28] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58] + +(90) Sort [codegen id : 32] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 + +(91) SortMergeJoin +Left keys [1]: [ws_bill_customer_sk#43] +Right keys [1]: [c_customer_sk#28] Join condition: None -(84) Project [codegen id : 36] -Output [3]: [ws_sold_date_sk#42, ws_quantity#45, ws_list_price#46] -Input [4]: [ws_sold_date_sk#42, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46] +(92) Project [codegen id : 34] +Output [3]: [ws_sold_date_sk#41, ws_quantity#44, ws_list_price#45] +Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -(85) ReusedExchange [Reuses operator id: 58] +(93) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#9] -(86) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ws_sold_date_sk#42] +(94) BroadcastHashJoin [codegen id : 34] +Left keys [1]: [ws_sold_date_sk#41] Right keys [1]: [d_date_sk#9] Join condition: None -(87) Project [codegen id : 36] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#55] -Input [4]: [ws_sold_date_sk#42, ws_quantity#45, ws_list_price#46, d_date_sk#9] +(95) Project [codegen id : 34] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#44 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#45 as decimal(12,2)))), DecimalType(18,2), true) AS sales#59] +Input [4]: [ws_sold_date_sk#41, ws_quantity#44, ws_list_price#45, d_date_sk#9] -(88) Union +(96) Union -(89) HashAggregate [codegen id : 37] -Input [1]: [sales#41] +(97) HashAggregate [codegen id : 35] +Input [1]: [sales#40] Keys: [] -Functions [1]: [partial_sum(sales#41)] -Aggregate Attributes [2]: [sum#56, isEmpty#57] -Results [2]: [sum#58, isEmpty#59] +Functions [1]: [partial_sum(sales#40)] +Aggregate Attributes [2]: [sum#60, isEmpty#61] +Results [2]: [sum#62, isEmpty#63] -(90) Exchange -Input [2]: [sum#58, isEmpty#59] -Arguments: SinglePartition, true, [id=#60] +(98) Exchange +Input [2]: [sum#62, isEmpty#63] +Arguments: SinglePartition, true, [id=#64] -(91) HashAggregate [codegen id : 38] -Input [2]: [sum#58, isEmpty#59] +(99) HashAggregate [codegen id : 36] +Input [2]: [sum#62, isEmpty#63] Keys: [] -Functions [1]: [sum(sales#41)] -Aggregate Attributes [1]: [sum(sales#41)#61] -Results [1]: [sum(sales#41)#61 AS sum(sales)#62] +Functions [1]: [sum(sales#40)] +Aggregate Attributes [1]: [sum(sales#40)#65] +Results [1]: [sum(sales#40)#65 AS sum(sales)#66] -(92) CollectLimit -Input [1]: [sum(sales)#62] +(100) CollectLimit +Input [1]: [sum(sales)#66] Arguments: 100 ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* HashAggregate (116) -+- Exchange (115) - +- * HashAggregate (114) - +- * HashAggregate (113) - +- * HashAggregate (112) - +- * Project (111) - +- * SortMergeJoin Inner (110) - :- * Sort (104) - : +- Exchange (103) - : +- * Project (102) - : +- * BroadcastHashJoin Inner BuildRight (101) - : :- * Filter (95) - : : +- * ColumnarToRow (94) - : : +- Scan parquet default.store_sales (93) - : +- BroadcastExchange (100) - : +- * Project (99) - : +- * Filter (98) - : +- * ColumnarToRow (97) - : +- Scan parquet default.date_dim (96) - +- * Sort (109) - +- Exchange (108) - +- * Filter (107) - +- * ColumnarToRow (106) - +- Scan parquet default.customer (105) - - -(93) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +Subquery:1 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#36, [id=#37] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- * Project (119) + +- * SortMergeJoin Inner (118) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Filter (103) + : : +- * ColumnarToRow (102) + : : +- Scan parquet default.store_sales (101) + : +- BroadcastExchange (108) + : +- * Project (107) + : +- * Filter (106) + : +- * ColumnarToRow (105) + : +- Scan parquet default.date_dim (104) + +- * Sort (117) + +- Exchange (116) + +- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet default.customer (113) + + +(101) Scan parquet default.store_sales +Output [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(102) ColumnarToRow [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(95) Filter [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Condition : (isnotnull(ss_customer_sk#25) AND isnotnull(ss_sold_date_sk#7)) +(103) Filter [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Condition : (isnotnull(ss_customer_sk#24) AND isnotnull(ss_sold_date_sk#7)) -(96) Scan parquet default.date_dim +(104) Scan parquet default.date_dim Output [2]: [d_date_sk#9, d_year#11] 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 -(97) ColumnarToRow [codegen id : 1] +(105) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] -(98) Filter [codegen id : 1] +(106) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] Condition : (d_year#11 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#9)) -(99) Project [codegen id : 1] +(107) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [2]: [d_date_sk#9, d_year#11] -(100) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#67] -(101) BroadcastHashJoin [codegen id : 2] +(109) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(102) Project [codegen id : 2] -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Input [5]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, d_date_sk#9] +(110) Project [codegen id : 2] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Input [5]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, d_date_sk#9] -(103) Exchange -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: hashpartitioning(ss_customer_sk#25, 5), true, [id=#64] +(111) Exchange +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#68] -(104) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(112) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(105) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +(113) Scan parquet default.customer +Output [1]: [c_customer_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(106) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#29] +(114) ColumnarToRow [codegen id : 4] +Input [1]: [c_customer_sk#28] -(107) Filter [codegen id : 4] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +(115) Filter [codegen id : 4] +Input [1]: [c_customer_sk#28] +Condition : isnotnull(c_customer_sk#28) -(108) Exchange -Input [1]: [c_customer_sk#29] -Arguments: hashpartitioning(c_customer_sk#29, 5), true, [id=#65] +(116) Exchange +Input [1]: [c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#69] -(109) Sort [codegen id : 5] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(117) Sort [codegen id : 5] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(110) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(118) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(111) Project [codegen id : 6] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(112) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#66, isEmpty#67] -Results [3]: [c_customer_sk#29, sum#68, isEmpty#69] - -(113) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#29, sum#68, isEmpty#69] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#70 AS csales#71] - -(114) HashAggregate [codegen id : 6] -Input [1]: [csales#71] +(119) Project [codegen id : 6] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(120) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#70, isEmpty#71] +Results [3]: [c_customer_sk#28, sum#72, isEmpty#73] + +(121) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#28, sum#72, isEmpty#73] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#74] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#74 AS csales#75] + +(122) HashAggregate [codegen id : 6] +Input [1]: [csales#75] Keys: [] -Functions [1]: [partial_max(csales#71)] -Aggregate Attributes [1]: [max#72] -Results [1]: [max#73] +Functions [1]: [partial_max(csales#75)] +Aggregate Attributes [1]: [max#76] +Results [1]: [max#77] -(115) Exchange -Input [1]: [max#73] -Arguments: SinglePartition, true, [id=#74] +(123) Exchange +Input [1]: [max#77] +Arguments: SinglePartition, true, [id=#78] -(116) HashAggregate [codegen id : 7] -Input [1]: [max#73] +(124) HashAggregate [codegen id : 7] +Input [1]: [max#77] Keys: [] -Functions [1]: [max(csales#71)] -Aggregate Attributes [1]: [max(csales#71)#75] -Results [1]: [max(csales#71)#75 AS tpcds_cmax#76] +Functions [1]: [max(csales#75)] +Aggregate Attributes [1]: [max(csales#75)#79] +Results [1]: [max(csales#75)#79 AS tpcds_cmax#80] -Subquery:2 Hosting operator id = 80 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:2 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index 9ee444cdd988c..695e6ccd71821 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -1,23 +1,23 @@ CollectLimit - WholeStageCodegen (38) + WholeStageCodegen (36) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 - WholeStageCodegen (37) + WholeStageCodegen (35) HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (18) + WholeStageCodegen (17) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - WholeStageCodegen (11) + WholeStageCodegen (10) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (10) + WholeStageCodegen (9) Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] InputAdapter SortMergeJoin [cs_item_sk,item_sk] @@ -30,48 +30,45 @@ CollectLimit ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] - WholeStageCodegen (9) + WholeStageCodegen (8) Sort [item_sk] - InputAdapter - Exchange [item_sk] #4 - WholeStageCodegen (8) - Project [item_sk] - Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [item_sk] + Filter [count(1)] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #5 - WholeStageCodegen (4) - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [d_date_sk,d_date] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_year] + Exchange [ss_item_sk] #4 + WholeStageCodegen (4) + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk,d_date] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - WholeStageCodegen (16) + Exchange [i_item_sk] #6 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] + WholeStageCodegen (15) 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))] @@ -79,7 +76,7 @@ CollectLimit WholeStageCodegen (7) HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - Exchange #10 + Exchange #9 WholeStageCodegen (6) HashAggregate [csales] [max,max] 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)),csales,sum,isEmpty] @@ -90,7 +87,7 @@ CollectLimit WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter - Exchange [ss_customer_sk] #11 + Exchange [ss_customer_sk] #10 WholeStageCodegen (2) Project [ss_customer_sk,ss_quantity,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -99,7 +96,7 @@ CollectLimit InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter - BroadcastExchange #12 + BroadcastExchange #11 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -110,7 +107,7 @@ CollectLimit WholeStageCodegen (5) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #13 + Exchange [c_customer_sk] #12 WholeStageCodegen (4) Filter [c_customer_sk] ColumnarToRow @@ -121,61 +118,75 @@ CollectLimit Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [ss_customer_sk] InputAdapter - Exchange [ss_customer_sk] #8 - WholeStageCodegen (12) + Exchange [ss_customer_sk] #7 + WholeStageCodegen (11) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (14) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #9 - WholeStageCodegen (14) + Exchange [c_customer_sk] #8 + WholeStageCodegen (13) Filter [c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) + BroadcastExchange #13 + WholeStageCodegen (16) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (36) + WholeStageCodegen (34) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - WholeStageCodegen (29) + WholeStageCodegen (27) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (28) + Exchange [ws_bill_customer_sk] #14 + WholeStageCodegen (26) Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] InputAdapter SortMergeJoin [ws_item_sk,item_sk] - WholeStageCodegen (20) + WholeStageCodegen (19) Sort [ws_item_sk] InputAdapter - Exchange [ws_item_sk] #16 - WholeStageCodegen (19) + Exchange [ws_item_sk] #15 + WholeStageCodegen (18) Filter [ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] - WholeStageCodegen (27) + WholeStageCodegen (25) Sort [item_sk] - InputAdapter - ReusedExchange [item_sk] #4 - WholeStageCodegen (34) + Project [item_sk] + Filter [count(1)] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (22) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,d_date] #4 + InputAdapter + WholeStageCodegen (24) + Sort [i_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_desc] #6 + 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))] @@ -185,14 +196,14 @@ CollectLimit Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (29) Sort [ss_customer_sk] InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 InputAdapter - WholeStageCodegen (33) + WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_sk] #9 + ReusedExchange [c_customer_sk] #8 InputAdapter - ReusedExchange [d_date_sk] #14 + ReusedExchange [d_date_sk] #13 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 51b85142f37ff..9a4c2b064d091 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -1,134 +1,140 @@ == Physical Plan == -TakeOrderedAndProject (130) -+- Union (129) - :- * HashAggregate (82) - : +- Exchange (81) - : +- * HashAggregate (80) - : +- * Project (79) - : +- * SortMergeJoin Inner (78) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- SortMergeJoin LeftSemi (52) - : : : :- * Sort (34) - : : : : +- Exchange (33) - : : : : +- * Project (32) - : : : : +- SortMergeJoin LeftSemi (31) +TakeOrderedAndProject (136) ++- Union (135) + :- * HashAggregate (80) + : +- Exchange (79) + : +- * HashAggregate (78) + : +- * Project (77) + : +- * SortMergeJoin Inner (76) + : :- * Project (58) + : : +- * BroadcastHashJoin Inner BuildRight (57) + : : :- SortMergeJoin LeftSemi (51) + : : : :- * Sort (33) + : : : : +- Exchange (32) + : : : : +- * Project (31) + : : : : +- SortMergeJoin LeftSemi (30) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Project (28) - : : : : +- * Filter (27) - : : : : +- * HashAggregate (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * SortMergeJoin Inner (23) - : : : : :- * Sort (17) - : : : : : +- Exchange (16) - : : : : : +- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Filter (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- Scan parquet default.store_sales (6) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Project (12) - : : : : : +- * Filter (11) - : : : : : +- * ColumnarToRow (10) - : : : : : +- Scan parquet default.date_dim (9) - : : : : +- * Sort (22) - : : : : +- Exchange (21) - : : : : +- * Filter (20) - : : : : +- * ColumnarToRow (19) - : : : : +- Scan parquet default.item (18) - : : : +- * Sort (51) - : : : +- * Project (50) - : : : +- * Filter (49) - : : : +- * HashAggregate (48) - : : : +- * HashAggregate (47) - : : : +- * Project (46) - : : : +- * SortMergeJoin Inner (45) - : : : :- * Sort (39) - : : : : +- Exchange (38) - : : : : +- * Filter (37) - : : : : +- * ColumnarToRow (36) - : : : : +- Scan parquet default.store_sales (35) - : : : +- * Sort (44) - : : : +- Exchange (43) - : : : +- * Filter (42) - : : : +- * ColumnarToRow (41) - : : : +- Scan parquet default.customer (40) - : : +- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet default.date_dim (53) - : +- SortMergeJoin LeftSemi (77) - : :- * Sort (64) - : : +- Exchange (63) - : : +- * Filter (62) - : : +- * ColumnarToRow (61) - : : +- Scan parquet default.customer (60) - : +- * Sort (76) - : +- Exchange (75) - : +- * Project (74) - : +- * Filter (73) - : +- * HashAggregate (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * SortMergeJoin Inner (69) - : :- * Sort (66) - : : +- ReusedExchange (65) - : +- * Sort (68) - : +- ReusedExchange (67) - +- * HashAggregate (128) - +- Exchange (127) - +- * HashAggregate (126) - +- * Project (125) - +- * SortMergeJoin Inner (124) - :- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- SortMergeJoin LeftSemi (105) - : : :- * Sort (93) - : : : +- Exchange (92) - : : : +- * Project (91) - : : : +- SortMergeJoin LeftSemi (90) - : : : :- * Sort (87) - : : : : +- Exchange (86) - : : : : +- * Filter (85) - : : : : +- * ColumnarToRow (84) - : : : : +- Scan parquet default.web_sales (83) - : : : +- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (104) - : : +- * Project (103) - : : +- * Filter (102) - : : +- * HashAggregate (101) - : : +- * HashAggregate (100) - : : +- * Project (99) - : : +- * SortMergeJoin Inner (98) - : : :- * Sort (95) - : : : +- ReusedExchange (94) - : : +- * Sort (97) - : : +- ReusedExchange (96) - : +- ReusedExchange (106) - +- SortMergeJoin LeftSemi (123) - :- * Sort (110) - : +- ReusedExchange (109) - +- * Sort (122) - +- Exchange (121) - +- * Project (120) - +- * Filter (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- * Project (116) - +- * SortMergeJoin Inner (115) - :- * Sort (112) - : +- ReusedExchange (111) - +- * Sort (114) - +- ReusedExchange (113) + : : : : +- * Sort (29) + : : : : +- * Project (28) + : : : : +- * Filter (27) + : : : : +- * HashAggregate (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * SortMergeJoin Inner (23) + : : : : :- * Sort (17) + : : : : : +- Exchange (16) + : : : : : +- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Filter (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- Scan parquet default.store_sales (6) + : : : : : +- BroadcastExchange (13) + : : : : : +- * Project (12) + : : : : : +- * Filter (11) + : : : : : +- * ColumnarToRow (10) + : : : : : +- Scan parquet default.date_dim (9) + : : : : +- * Sort (22) + : : : : +- Exchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.item (18) + : : : +- * Sort (50) + : : : +- * Project (49) + : : : +- * Filter (48) + : : : +- * HashAggregate (47) + : : : +- * HashAggregate (46) + : : : +- * Project (45) + : : : +- * SortMergeJoin Inner (44) + : : : :- * Sort (38) + : : : : +- Exchange (37) + : : : : +- * Filter (36) + : : : : +- * ColumnarToRow (35) + : : : : +- Scan parquet default.store_sales (34) + : : : +- * Sort (43) + : : : +- Exchange (42) + : : : +- * Filter (41) + : : : +- * ColumnarToRow (40) + : : : +- Scan parquet default.customer (39) + : : +- BroadcastExchange (56) + : : +- * Project (55) + : : +- * Filter (54) + : : +- * ColumnarToRow (53) + : : +- Scan parquet default.date_dim (52) + : +- SortMergeJoin LeftSemi (75) + : :- * Sort (63) + : : +- Exchange (62) + : : +- * Filter (61) + : : +- * ColumnarToRow (60) + : : +- Scan parquet default.customer (59) + : +- * Sort (74) + : +- * Project (73) + : +- * Filter (72) + : +- * HashAggregate (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * SortMergeJoin Inner (68) + : :- * Sort (65) + : : +- ReusedExchange (64) + : +- * Sort (67) + : +- ReusedExchange (66) + +- * HashAggregate (134) + +- Exchange (133) + +- * HashAggregate (132) + +- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- SortMergeJoin LeftSemi (112) + : : :- * Sort (100) + : : : +- Exchange (99) + : : : +- * Project (98) + : : : +- SortMergeJoin LeftSemi (97) + : : : :- * Sort (85) + : : : : +- Exchange (84) + : : : : +- * Filter (83) + : : : : +- * ColumnarToRow (82) + : : : : +- Scan parquet default.web_sales (81) + : : : +- * Sort (96) + : : : +- * Project (95) + : : : +- * Filter (94) + : : : +- * HashAggregate (93) + : : : +- * HashAggregate (92) + : : : +- * Project (91) + : : : +- * SortMergeJoin Inner (90) + : : : :- * Sort (87) + : : : : +- ReusedExchange (86) + : : : +- * Sort (89) + : : : +- ReusedExchange (88) + : : +- * Sort (111) + : : +- * Project (110) + : : +- * Filter (109) + : : +- * HashAggregate (108) + : : +- * HashAggregate (107) + : : +- * Project (106) + : : +- * SortMergeJoin Inner (105) + : : :- * Sort (102) + : : : +- ReusedExchange (101) + : : +- * Sort (104) + : : +- ReusedExchange (103) + : +- ReusedExchange (113) + +- SortMergeJoin LeftSemi (129) + :- * Sort (117) + : +- ReusedExchange (116) + +- * Sort (128) + +- * Project (127) + +- * Filter (126) + +- * HashAggregate (125) + +- * HashAggregate (124) + +- * Project (123) + +- * SortMergeJoin Inner (122) + :- * Sort (119) + : +- ReusedExchange (118) + +- * Sort (121) + +- ReusedExchange (120) (1) Scan parquet default.catalog_sales @@ -259,612 +265,642 @@ Condition : (count(1)#22 > 4) Output [1]: [item_sk#21] Input [2]: [item_sk#21, count(1)#22] -(29) Exchange -Input [1]: [item_sk#21] -Arguments: hashpartitioning(item_sk#21, 5), true, [id=#23] - -(30) Sort [codegen id : 9] +(29) Sort [codegen id : 8] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin +(30) SortMergeJoin Left keys [1]: [cs_item_sk#3] Right keys [1]: [item_sk#21] Join condition: None -(32) Project [codegen id : 10] +(31) Project [codegen id : 9] Output [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -(33) Exchange +(32) Exchange Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#24] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#23] -(34) Sort [codegen id : 11] +(33) Sort [codegen id : 10] Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 -(35) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(34) Scan parquet default.store_sales +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 12] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(35) ColumnarToRow [codegen id : 11] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(37) Filter [codegen id : 12] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Condition : isnotnull(ss_customer_sk#25) +(36) Filter [codegen id : 11] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Condition : isnotnull(ss_customer_sk#24) -(38) Exchange -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: hashpartitioning(ss_customer_sk#25, 5), true, [id=#28] +(37) Exchange +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#27] -(39) Sort [codegen id : 13] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(38) Sort [codegen id : 12] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(40) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +(39) Scan parquet default.customer +Output [1]: [c_customer_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 14] -Input [1]: [c_customer_sk#29] +(40) ColumnarToRow [codegen id : 13] +Input [1]: [c_customer_sk#28] -(42) Filter [codegen id : 14] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +(41) Filter [codegen id : 13] +Input [1]: [c_customer_sk#28] +Condition : isnotnull(c_customer_sk#28) -(43) Exchange -Input [1]: [c_customer_sk#29] -Arguments: hashpartitioning(c_customer_sk#29, 5), true, [id=#30] +(42) Exchange +Input [1]: [c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#29] -(44) Sort [codegen id : 15] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(43) Sort [codegen id : 14] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 16] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(44) SortMergeJoin [codegen id : 15] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(46) Project [codegen id : 16] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(47) HashAggregate [codegen id : 16] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(48) HashAggregate [codegen id : 16] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#35] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#35 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] - -(49) Filter [codegen id : 16] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) - -(50) Project [codegen id : 16] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] - -(51) Sort [codegen id : 16] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 - -(52) SortMergeJoin +(45) Project [codegen id : 15] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(46) HashAggregate [codegen id : 15] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#30, isEmpty#31] +Results [3]: [c_customer_sk#28, sum#32, isEmpty#33] + +(47) HashAggregate [codegen id : 15] +Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] + +(48) Filter [codegen id : 15] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) + +(49) Project [codegen id : 15] +Output [1]: [c_customer_sk#28] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] + +(50) Sort [codegen id : 15] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#28] Join condition: None -(53) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#11, d_moy#39] +(52) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#11, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 17] -Input [3]: [d_date_sk#9, d_year#11, d_moy#39] +(53) ColumnarToRow [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#38] -(55) Filter [codegen id : 17] -Input [3]: [d_date_sk#9, d_year#11, d_moy#39] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#39)) AND (d_year#11 = 2000)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#9)) +(54) Filter [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#38] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#38)) AND (d_year#11 = 2000)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#9)) -(56) Project [codegen id : 17] +(55) Project [codegen id : 16] Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#11, d_moy#39] +Input [3]: [d_date_sk#9, d_year#11, d_moy#38] -(57) BroadcastExchange +(56) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] -(58) BroadcastHashJoin [codegen id : 18] +(57) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#1] Right keys [1]: [d_date_sk#9] Join condition: None -(59) Project [codegen id : 18] +(58) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, d_date_sk#9] -(60) Scan parquet default.customer -Output [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] +(59) Scan parquet default.customer +Output [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 19] -Input [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] +(60) ColumnarToRow [codegen id : 18] +Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] -(62) Filter [codegen id : 19] -Input [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] -Condition : isnotnull(c_customer_sk#29) +(61) Filter [codegen id : 18] +Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +Condition : isnotnull(c_customer_sk#28) -(63) Exchange -Input [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] -Arguments: hashpartitioning(c_customer_sk#29, 5), true, [id=#43] +(62) Exchange +Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#42] -(64) Sort [codegen id : 20] -Input [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(63) Sort [codegen id : 19] +Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(65) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(64) ReusedExchange [Reuses operator id: 37] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(66) Sort [codegen id : 22] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(65) Sort [codegen id : 21] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(67) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#29] +(66) ReusedExchange [Reuses operator id: 42] +Output [1]: [c_customer_sk#28] -(68) Sort [codegen id : 24] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 23] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(68) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(70) Project [codegen id : 25] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(71) HashAggregate [codegen id : 25] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(72) HashAggregate [codegen id : 25] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#35] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#35 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] - -(73) Filter [codegen id : 25] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) - -(74) Project [codegen id : 25] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#44] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#36] - -(75) Exchange -Input [1]: [c_customer_sk#29#44] -Arguments: hashpartitioning(c_customer_sk#29#44, 5), true, [id=#45] - -(76) Sort [codegen id : 26] -Input [1]: [c_customer_sk#29#44] -Arguments: [c_customer_sk#29#44 ASC NULLS FIRST], false, 0 - -(77) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#44] +(69) Project [codegen id : 24] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(70) HashAggregate [codegen id : 24] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#30, isEmpty#31] +Results [3]: [c_customer_sk#28, sum#32, isEmpty#33] + +(71) HashAggregate [codegen id : 24] +Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] + +(72) Filter [codegen id : 24] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) + +(73) Project [codegen id : 24] +Output [1]: [c_customer_sk#28 AS c_customer_sk#28#43] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] + +(74) Sort [codegen id : 24] +Input [1]: [c_customer_sk#28#43] +Arguments: [c_customer_sk#28#43 ASC NULLS FIRST], false, 0 + +(75) SortMergeJoin +Left keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#28#43] Join condition: None -(78) SortMergeJoin [codegen id : 27] +(76) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#28] Join condition: None -(79) Project [codegen id : 27] -Output [4]: [cs_quantity#4, cs_list_price#5, c_first_name#41, c_last_name#42] -Input [6]: [cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, c_customer_sk#29, c_first_name#41, c_last_name#42] +(77) Project [codegen id : 25] +Output [4]: [cs_quantity#4, cs_list_price#5, c_first_name#40, c_last_name#41] +Input [6]: [cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, c_customer_sk#28, c_first_name#40, c_last_name#41] -(80) HashAggregate [codegen id : 27] -Input [4]: [cs_quantity#4, cs_list_price#5, c_first_name#41, c_last_name#42] -Keys [2]: [c_last_name#42, c_first_name#41] +(78) HashAggregate [codegen id : 25] +Input [4]: [cs_quantity#4, cs_list_price#5, c_first_name#40, c_last_name#41] +Keys [2]: [c_last_name#41, c_first_name#40] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [4]: [c_last_name#42, c_first_name#41, sum#48, isEmpty#49] +Aggregate Attributes [2]: [sum#44, isEmpty#45] +Results [4]: [c_last_name#41, c_first_name#40, sum#46, isEmpty#47] -(81) Exchange -Input [4]: [c_last_name#42, c_first_name#41, sum#48, isEmpty#49] -Arguments: hashpartitioning(c_last_name#42, c_first_name#41, 5), true, [id=#50] +(79) Exchange +Input [4]: [c_last_name#41, c_first_name#40, sum#46, isEmpty#47] +Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), true, [id=#48] -(82) HashAggregate [codegen id : 28] -Input [4]: [c_last_name#42, c_first_name#41, sum#48, isEmpty#49] -Keys [2]: [c_last_name#42, c_first_name#41] +(80) HashAggregate [codegen id : 26] +Input [4]: [c_last_name#41, c_first_name#40, sum#46, isEmpty#47] +Keys [2]: [c_last_name#41, c_first_name#40] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [3]: [c_last_name#42, c_first_name#41, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sales#52] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#49] +Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#49 AS sales#50] -(83) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] +(81) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 29] -Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] +(82) ColumnarToRow [codegen id : 27] +Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] + +(83) Filter [codegen id : 27] +Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Condition : (isnotnull(ws_bill_customer_sk#53) AND isnotnull(ws_sold_date_sk#51)) + +(84) Exchange +Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Arguments: hashpartitioning(ws_item_sk#52, 5), true, [id=#56] -(85) Filter [codegen id : 29] -Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Condition : (isnotnull(ws_bill_customer_sk#55) AND isnotnull(ws_sold_date_sk#53)) +(85) Sort [codegen id : 28] +Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Arguments: [ws_item_sk#52 ASC NULLS FIRST], false, 0 -(86) Exchange -Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Arguments: hashpartitioning(ws_item_sk#54, 5), true, [id=#58] +(86) ReusedExchange [Reuses operator id: 16] +Output [2]: [ss_item_sk#8, d_date#10] + +(87) Sort [codegen id : 31] +Input [2]: [ss_item_sk#8, d_date#10] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 + +(88) ReusedExchange [Reuses operator id: 21] +Output [2]: [i_item_sk#14, i_item_desc#15] + +(89) Sort [codegen id : 33] +Input [2]: [i_item_sk#14, i_item_desc#15] +Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 + +(90) SortMergeJoin [codegen id : 34] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#14] +Join condition: None + +(91) Project [codegen id : 34] +Output [3]: [d_date#10, i_item_sk#14, i_item_desc#15] +Input [4]: [ss_item_sk#8, d_date#10, i_item_sk#14, i_item_desc#15] + +(92) HashAggregate [codegen id : 34] +Input [3]: [d_date#10, i_item_sk#14, i_item_desc#15] +Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#58] +Results [4]: [substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10, count#59] + +(93) HashAggregate [codegen id : 34] +Input [4]: [substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10, count#59] +Keys [3]: [substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#60] +Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#60 AS count(1)#61] -(87) Sort [codegen id : 30] -Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Arguments: [ws_item_sk#54 ASC NULLS FIRST], false, 0 +(94) Filter [codegen id : 34] +Input [2]: [item_sk#21, count(1)#61] +Condition : (count(1)#61 > 4) -(88) ReusedExchange [Reuses operator id: 29] +(95) Project [codegen id : 34] Output [1]: [item_sk#21] +Input [2]: [item_sk#21, count(1)#61] -(89) Sort [codegen id : 37] +(96) Sort [codegen id : 34] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin -Left keys [1]: [ws_item_sk#54] +(97) SortMergeJoin +Left keys [1]: [ws_item_sk#52] Right keys [1]: [item_sk#21] Join condition: None -(91) Project [codegen id : 38] -Output [4]: [ws_sold_date_sk#53, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Input [5]: [ws_sold_date_sk#53, ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] +(98) Project [codegen id : 35] +Output [4]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -(92) Exchange -Input [4]: [ws_sold_date_sk#53, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Arguments: hashpartitioning(ws_bill_customer_sk#55, 5), true, [id=#59] +(99) Exchange +Input [4]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), true, [id=#62] -(93) Sort [codegen id : 39] -Input [4]: [ws_sold_date_sk#53, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Arguments: [ws_bill_customer_sk#55 ASC NULLS FIRST], false, 0 +(100) Sort [codegen id : 36] +Input [4]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 -(94) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(101) ReusedExchange [Reuses operator id: 37] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(95) Sort [codegen id : 41] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(102) Sort [codegen id : 38] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(96) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#29] +(103) ReusedExchange [Reuses operator id: 42] +Output [1]: [c_customer_sk#28] -(97) Sort [codegen id : 43] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(104) Sort [codegen id : 40] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin [codegen id : 44] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(105) SortMergeJoin [codegen id : 41] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(99) Project [codegen id : 44] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(100) HashAggregate [codegen id : 44] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#60, isEmpty#61] -Results [3]: [c_customer_sk#29, sum#62, isEmpty#63] - -(101) HashAggregate [codegen id : 44] -Input [3]: [c_customer_sk#29, sum#62, isEmpty#63] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#64] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(102) Filter [codegen id : 44] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) - -(103) Project [codegen id : 44] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(104) Sort [codegen id : 44] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 - -(105) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +(106) Project [codegen id : 41] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(107) HashAggregate [codegen id : 41] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [3]: [c_customer_sk#28, sum#65, isEmpty#66] + +(108) HashAggregate [codegen id : 41] +Input [3]: [c_customer_sk#28, sum#65, isEmpty#66] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] + +(109) Filter [codegen id : 41] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) + +(110) Project [codegen id : 41] +Output [1]: [c_customer_sk#28] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] + +(111) Sort [codegen id : 41] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 + +(112) SortMergeJoin +Left keys [1]: [ws_bill_customer_sk#53] +Right keys [1]: [c_customer_sk#28] Join condition: None -(106) ReusedExchange [Reuses operator id: 57] +(113) ReusedExchange [Reuses operator id: 56] Output [1]: [d_date_sk#9] -(107) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#53] +(114) BroadcastHashJoin [codegen id : 43] +Left keys [1]: [ws_sold_date_sk#51] Right keys [1]: [d_date_sk#9] Join condition: None -(108) Project [codegen id : 46] -Output [3]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57] -Input [5]: [ws_sold_date_sk#53, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, d_date_sk#9] +(115) Project [codegen id : 43] +Output [3]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +Input [5]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, d_date_sk#9] -(109) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] +(116) ReusedExchange [Reuses operator id: 62] +Output [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] -(110) Sort [codegen id : 48] -Input [3]: [c_customer_sk#29, c_first_name#41, c_last_name#42] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(117) Sort [codegen id : 45] +Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(111) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(118) ReusedExchange [Reuses operator id: 37] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(112) Sort [codegen id : 50] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(119) Sort [codegen id : 47] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(113) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#29] +(120) ReusedExchange [Reuses operator id: 42] +Output [1]: [c_customer_sk#28] -(114) Sort [codegen id : 52] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(121) Sort [codegen id : 49] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(115) SortMergeJoin [codegen id : 53] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(122) SortMergeJoin [codegen id : 50] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(116) Project [codegen id : 53] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(117) HashAggregate [codegen id : 53] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#60, isEmpty#61] -Results [3]: [c_customer_sk#29, sum#62, isEmpty#63] - -(118) HashAggregate [codegen id : 53] -Input [3]: [c_customer_sk#29, sum#62, isEmpty#63] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#64] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(119) Filter [codegen id : 53] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) - -(120) Project [codegen id : 53] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#66] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(121) Exchange -Input [1]: [c_customer_sk#29#66] -Arguments: hashpartitioning(c_customer_sk#29#66, 5), true, [id=#67] - -(122) Sort [codegen id : 54] -Input [1]: [c_customer_sk#29#66] -Arguments: [c_customer_sk#29#66 ASC NULLS FIRST], false, 0 - -(123) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#66] +(123) Project [codegen id : 50] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(124) HashAggregate [codegen id : 50] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [3]: [c_customer_sk#28, sum#65, isEmpty#66] + +(125) HashAggregate [codegen id : 50] +Input [3]: [c_customer_sk#28, sum#65, isEmpty#66] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] + +(126) Filter [codegen id : 50] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) + +(127) Project [codegen id : 50] +Output [1]: [c_customer_sk#28 AS c_customer_sk#28#69] +Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] + +(128) Sort [codegen id : 50] +Input [1]: [c_customer_sk#28#69] +Arguments: [c_customer_sk#28#69 ASC NULLS FIRST], false, 0 + +(129) SortMergeJoin +Left keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#28#69] Join condition: None -(124) SortMergeJoin [codegen id : 55] -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +(130) SortMergeJoin [codegen id : 51] +Left keys [1]: [ws_bill_customer_sk#53] +Right keys [1]: [c_customer_sk#28] Join condition: None -(125) Project [codegen id : 55] -Output [4]: [ws_quantity#56, ws_list_price#57, c_first_name#41, c_last_name#42] -Input [6]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, c_customer_sk#29, c_first_name#41, c_last_name#42] +(131) Project [codegen id : 51] +Output [4]: [ws_quantity#54, ws_list_price#55, c_first_name#40, c_last_name#41] +Input [6]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, c_customer_sk#28, c_first_name#40, c_last_name#41] -(126) HashAggregate [codegen id : 55] -Input [4]: [ws_quantity#56, ws_list_price#57, c_first_name#41, c_last_name#42] -Keys [2]: [c_last_name#42, c_first_name#41] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [4]: [c_last_name#42, c_first_name#41, sum#70, isEmpty#71] +(132) HashAggregate [codegen id : 51] +Input [4]: [ws_quantity#54, ws_list_price#55, c_first_name#40, c_last_name#41] +Keys [2]: [c_last_name#41, c_first_name#40] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#70, isEmpty#71] +Results [4]: [c_last_name#41, c_first_name#40, sum#72, isEmpty#73] -(127) Exchange -Input [4]: [c_last_name#42, c_first_name#41, sum#70, isEmpty#71] -Arguments: hashpartitioning(c_last_name#42, c_first_name#41, 5), true, [id=#72] +(133) Exchange +Input [4]: [c_last_name#41, c_first_name#40, sum#72, isEmpty#73] +Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), true, [id=#74] -(128) HashAggregate [codegen id : 56] -Input [4]: [c_last_name#42, c_first_name#41, sum#70, isEmpty#71] -Keys [2]: [c_last_name#42, c_first_name#41] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#73] -Results [3]: [c_last_name#42, c_first_name#41, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#73 AS sales#74] +(134) HashAggregate [codegen id : 52] +Input [4]: [c_last_name#41, c_first_name#40, sum#72, isEmpty#73] +Keys [2]: [c_last_name#41, c_first_name#40] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#75] +Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#75 AS sales#76] -(129) Union +(135) Union -(130) TakeOrderedAndProject -Input [3]: [c_last_name#42, c_first_name#41, sales#52] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#52 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#52] +(136) TakeOrderedAndProject +Input [3]: [c_last_name#41, c_first_name#40, sales#50] +Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* HashAggregate (154) -+- Exchange (153) - +- * HashAggregate (152) - +- * HashAggregate (151) - +- * HashAggregate (150) - +- * Project (149) - +- * SortMergeJoin Inner (148) - :- * Sort (142) - : +- Exchange (141) - : +- * Project (140) - : +- * BroadcastHashJoin Inner BuildRight (139) - : :- * Filter (133) - : : +- * ColumnarToRow (132) - : : +- Scan parquet default.store_sales (131) - : +- BroadcastExchange (138) - : +- * Project (137) - : +- * Filter (136) - : +- * ColumnarToRow (135) - : +- Scan parquet default.date_dim (134) - +- * Sort (147) - +- Exchange (146) - +- * Filter (145) - +- * ColumnarToRow (144) - +- Scan parquet default.customer (143) - - -(131) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +Subquery:1 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#36, [id=#37] +* HashAggregate (160) ++- Exchange (159) + +- * HashAggregate (158) + +- * HashAggregate (157) + +- * HashAggregate (156) + +- * Project (155) + +- * SortMergeJoin Inner (154) + :- * Sort (148) + : +- Exchange (147) + : +- * Project (146) + : +- * BroadcastHashJoin Inner BuildRight (145) + : :- * Filter (139) + : : +- * ColumnarToRow (138) + : : +- Scan parquet default.store_sales (137) + : +- BroadcastExchange (144) + : +- * Project (143) + : +- * Filter (142) + : +- * ColumnarToRow (141) + : +- Scan parquet default.date_dim (140) + +- * Sort (153) + +- Exchange (152) + +- * Filter (151) + +- * ColumnarToRow (150) + +- Scan parquet default.customer (149) + + +(137) Scan parquet default.store_sales +Output [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] +(138) ColumnarToRow [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(133) Filter [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Condition : (isnotnull(ss_customer_sk#25) AND isnotnull(ss_sold_date_sk#7)) +(139) Filter [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Condition : (isnotnull(ss_customer_sk#24) AND isnotnull(ss_sold_date_sk#7)) -(134) Scan parquet default.date_dim +(140) Scan parquet default.date_dim Output [2]: [d_date_sk#9, d_year#11] 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 -(135) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] -(136) Filter [codegen id : 1] +(142) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] Condition : (d_year#11 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#9)) -(137) Project [codegen id : 1] +(143) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [2]: [d_date_sk#9, d_year#11] -(138) BroadcastExchange +(144) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] -(139) BroadcastHashJoin [codegen id : 2] +(145) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(140) Project [codegen id : 2] -Output [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Input [5]: [ss_sold_date_sk#7, ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, d_date_sk#9] +(146) Project [codegen id : 2] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Input [5]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, d_date_sk#9] -(141) Exchange -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: hashpartitioning(ss_customer_sk#25, 5), true, [id=#76] +(147) Exchange +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#78] -(142) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27] -Arguments: [ss_customer_sk#25 ASC NULLS FIRST], false, 0 +(148) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(143) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +(149) Scan parquet default.customer +Output [1]: [c_customer_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(144) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#29] +(150) ColumnarToRow [codegen id : 4] +Input [1]: [c_customer_sk#28] -(145) Filter [codegen id : 4] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +(151) Filter [codegen id : 4] +Input [1]: [c_customer_sk#28] +Condition : isnotnull(c_customer_sk#28) -(146) Exchange -Input [1]: [c_customer_sk#29] -Arguments: hashpartitioning(c_customer_sk#29, 5), true, [id=#77] +(152) Exchange +Input [1]: [c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#79] -(147) Sort [codegen id : 5] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +(153) Sort [codegen id : 5] +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(148) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#25] -Right keys [1]: [c_customer_sk#29] +(154) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#28] Join condition: None -(149) Project [codegen id : 6] -Output [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Input [4]: [ss_customer_sk#25, ss_quantity#26, ss_sales_price#27, c_customer_sk#29] - -(150) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#26, ss_sales_price#27, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#29, sum#80, isEmpty#81] - -(151) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#29, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#26 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#27 as decimal(12,2)))), DecimalType(18,2), true))#82 AS csales#83] - -(152) HashAggregate [codegen id : 6] -Input [1]: [csales#83] +(155) Project [codegen id : 6] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] + +(156) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] +Keys [1]: [c_customer_sk#28] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#80, isEmpty#81] +Results [3]: [c_customer_sk#28, sum#82, isEmpty#83] + +(157) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#28, sum#82, isEmpty#83] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#84] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#84 AS csales#85] + +(158) HashAggregate [codegen id : 6] +Input [1]: [csales#85] Keys: [] -Functions [1]: [partial_max(csales#83)] -Aggregate Attributes [1]: [max#84] -Results [1]: [max#85] +Functions [1]: [partial_max(csales#85)] +Aggregate Attributes [1]: [max#86] +Results [1]: [max#87] -(153) Exchange -Input [1]: [max#85] -Arguments: SinglePartition, true, [id=#86] +(159) Exchange +Input [1]: [max#87] +Arguments: SinglePartition, true, [id=#88] -(154) HashAggregate [codegen id : 7] -Input [1]: [max#85] +(160) HashAggregate [codegen id : 7] +Input [1]: [max#87] Keys: [] -Functions [1]: [max(csales#83)] -Aggregate Attributes [1]: [max(csales#83)#87] -Results [1]: [max(csales#83)#87 AS tpcds_cmax#88] +Functions [1]: [max(csales#85)] +Aggregate Attributes [1]: [max(csales#85)#89] +Results [1]: [max(csales#85)#89 AS tpcds_cmax#90] -Subquery:2 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:2 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] -Subquery:3 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:3 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] -Subquery:4 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:4 Hosting operator id = 126 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index e8891f032a091..4279bf3e16a82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (28) + WholeStageCodegen (26) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [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)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (27) + WholeStageCodegen (25) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (17) Project [cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - WholeStageCodegen (11) + WholeStageCodegen (10) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (10) + WholeStageCodegen (9) Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] InputAdapter SortMergeJoin [cs_item_sk,item_sk] @@ -31,48 +31,45 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] - WholeStageCodegen (9) + WholeStageCodegen (8) Sort [item_sk] - InputAdapter - Exchange [item_sk] #4 - WholeStageCodegen (8) - Project [item_sk] - Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [item_sk] + Filter [count(1)] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #5 - WholeStageCodegen (4) - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [d_date_sk,d_date] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_year] + Exchange [ss_item_sk] #4 + WholeStageCodegen (4) + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk,d_date] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - WholeStageCodegen (16) + Exchange [i_item_sk] #6 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] + WholeStageCodegen (15) 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))] @@ -80,7 +77,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - Exchange #10 + Exchange #9 WholeStageCodegen (6) HashAggregate [csales] [max,max] 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)),csales,sum,isEmpty] @@ -91,7 +88,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter - Exchange [ss_customer_sk] #11 + Exchange [ss_customer_sk] #10 WholeStageCodegen (2) Project [ss_customer_sk,ss_quantity,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -100,7 +97,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter - BroadcastExchange #12 + BroadcastExchange #11 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -111,7 +108,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #13 + Exchange [c_customer_sk] #12 WholeStageCodegen (4) Filter [c_customer_sk] ColumnarToRow @@ -122,28 +119,28 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [ss_customer_sk] InputAdapter - Exchange [ss_customer_sk] #8 - WholeStageCodegen (12) + Exchange [ss_customer_sk] #7 + WholeStageCodegen (11) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (14) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #9 - WholeStageCodegen (14) + Exchange [c_customer_sk] #8 + WholeStageCodegen (13) Filter [c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (17) + BroadcastExchange #13 + WholeStageCodegen (16) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -151,73 +148,84 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter SortMergeJoin [c_customer_sk,c_customer_sk] - WholeStageCodegen (20) + WholeStageCodegen (19) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #15 - WholeStageCodegen (19) + Exchange [c_customer_sk] #14 + WholeStageCodegen (18) Filter [c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (26) + WholeStageCodegen (24) Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #16 - WholeStageCodegen (25) - 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] #1 - 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] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,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] #1 + 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] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (21) + Sort [ss_customer_sk] InputAdapter - WholeStageCodegen (22) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 + InputAdapter + WholeStageCodegen (23) + Sort [c_customer_sk] InputAdapter - WholeStageCodegen (24) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #9 - WholeStageCodegen (56) + ReusedExchange [c_customer_sk] #8 + WholeStageCodegen (52) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [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)),sales,sum,isEmpty] InputAdapter - Exchange [c_last_name,c_first_name] #17 - WholeStageCodegen (55) + Exchange [c_last_name,c_first_name] #15 + WholeStageCodegen (51) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (46) + WholeStageCodegen (43) Project [ws_bill_customer_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - WholeStageCodegen (39) + WholeStageCodegen (36) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #18 - WholeStageCodegen (38) + Exchange [ws_bill_customer_sk] #16 + WholeStageCodegen (35) Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] InputAdapter SortMergeJoin [ws_item_sk,item_sk] - WholeStageCodegen (30) + WholeStageCodegen (28) Sort [ws_item_sk] InputAdapter - Exchange [ws_item_sk] #19 - WholeStageCodegen (29) + Exchange [ws_item_sk] #17 + WholeStageCodegen (27) Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] - WholeStageCodegen (37) + WholeStageCodegen (34) Sort [item_sk] - InputAdapter - ReusedExchange [item_sk] #4 - WholeStageCodegen (44) + Project [item_sk] + Filter [count(1)] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (31) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,d_date] #4 + InputAdapter + WholeStageCodegen (33) + Sort [i_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_desc] #6 + WholeStageCodegen (41) 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))] @@ -227,42 +235,39 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (41) + WholeStageCodegen (38) Sort [ss_customer_sk] InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (40) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_sk] #9 + ReusedExchange [c_customer_sk] #8 InputAdapter - ReusedExchange [d_date_sk] #14 + ReusedExchange [d_date_sk] #13 InputAdapter SortMergeJoin [c_customer_sk,c_customer_sk] - WholeStageCodegen (48) + WholeStageCodegen (45) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - WholeStageCodegen (54) + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 + WholeStageCodegen (50) Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #20 - WholeStageCodegen (53) - 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] #1 - 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] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,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] #1 + 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] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (47) + Sort [ss_customer_sk] InputAdapter - WholeStageCodegen (50) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 + InputAdapter + WholeStageCodegen (49) + Sort [c_customer_sk] InputAdapter - WholeStageCodegen (52) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #9 + ReusedExchange [c_customer_sk] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index eae118d46245d..7fec07e259559 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -1,67 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (63) -+- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * HashAggregate (59) - +- Exchange (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- SortMergeJoin LeftSemi (35) - : : : :- * Sort (19) - : : : : +- Exchange (18) - : : : : +- SortMergeJoin LeftSemi (17) +TakeOrderedAndProject (61) ++- * HashAggregate (60) + +- Exchange (59) + +- * HashAggregate (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- SortMergeJoin LeftSemi (33) + : : : :- * Sort (17) + : : : : +- Exchange (16) + : : : : +- SortMergeJoin LeftSemi (15) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (10) - : : : : : +- Exchange (9) - : : : : : +- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.web_sales (6) - : : : : +- * Sort (12) - : : : : +- ReusedExchange (11) - : : : +- * Project (34) - : : : +- * SortMergeJoin Inner (33) - : : : :- * Sort (27) - : : : : +- Exchange (26) - : : : : +- * Project (25) - : : : : +- * SortMergeJoin Inner (24) - : : : : :- * Sort (21) - : : : : : +- ReusedExchange (20) - : : : : +- * Sort (23) - : : : : +- ReusedExchange (22) - : : : +- * Sort (32) - : : : +- Exchange (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.web_returns (28) - : : +- BroadcastExchange (40) - : : +- * Project (39) - : : +- * Filter (38) - : : +- * ColumnarToRow (37) - : : +- Scan parquet default.customer_address (36) - : +- BroadcastExchange (47) - : +- * Project (46) - : +- * Filter (45) - : +- * ColumnarToRow (44) - : +- Scan parquet default.web_site (43) - +- BroadcastExchange (54) - +- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) + : : : : +- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (10) + : : : : : +- Exchange (9) + : : : : : +- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.web_sales (6) + : : : : +- * Sort (12) + : : : : +- ReusedExchange (11) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * SortMergeJoin Inner (22) + : : : : :- * Sort (19) + : : : : : +- ReusedExchange (18) + : : : : +- * Sort (21) + : : : : +- ReusedExchange (20) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.web_returns (26) + : : +- BroadcastExchange (38) + : : +- * Project (37) + : : +- * Filter (36) + : : +- * ColumnarToRow (35) + : : +- Scan parquet default.customer_address (34) + : +- BroadcastExchange (45) + : +- * Project (44) + : +- * Filter (43) + : +- * ColumnarToRow (42) + : +- Scan parquet default.web_site (41) + +- BroadcastExchange (52) + +- * Project (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet default.date_dim (48) (1) Scan parquet default.web_sales @@ -124,224 +122,216 @@ Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#10) Output [1]: [ws_order_number#4 AS ws_order_number#4#12] Input [4]: [ws_warehouse_sk#8, ws_order_number#4, ws_warehouse_sk#10, ws_order_number#11] -(15) Exchange -Input [1]: [ws_order_number#4#12] -Arguments: hashpartitioning(ws_order_number#4#12, 5), true, [id=#13] - -(16) Sort [codegen id : 8] -Input [1]: [ws_order_number#4#12] -Arguments: [ws_order_number#4#12 ASC NULLS FIRST], false, 0 - -(17) SortMergeJoin +(15) SortMergeJoin Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#4#12] Join condition: None -(18) Exchange +(16) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), true, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), true, [id=#13] -(19) Sort [codegen id : 9] +(17) Sort [codegen id : 8] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 -(20) ReusedExchange [Reuses operator id: 9] +(18) ReusedExchange [Reuses operator id: 9] Output [2]: [ws_warehouse_sk#8, ws_order_number#4] -(21) Sort [codegen id : 11] +(19) Sort [codegen id : 10] Input [2]: [ws_warehouse_sk#8, ws_order_number#4] Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 -(22) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] +(20) ReusedExchange [Reuses operator id: 9] +Output [2]: [ws_warehouse_sk#14, ws_order_number#15] -(23) Sort [codegen id : 13] -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 12] +Input [2]: [ws_warehouse_sk#14, ws_order_number#15] +Arguments: [ws_order_number#15 ASC NULLS FIRST], false, 0 -(24) SortMergeJoin [codegen id : 14] +(22) SortMergeJoin [codegen id : 13] Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#16] -Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#15) +Right keys [1]: [ws_order_number#15] +Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#14) -(25) Project [codegen id : 14] +(23) Project [codegen id : 13] Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#8, ws_order_number#4, ws_warehouse_sk#15, ws_order_number#16] +Input [4]: [ws_warehouse_sk#8, ws_order_number#4, ws_warehouse_sk#14, ws_order_number#15] -(26) Exchange +(24) Exchange Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), true, [id=#17] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), true, [id=#16] -(27) Sort [codegen id : 15] +(25) Sort [codegen id : 14] Input [1]: [ws_order_number#4] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 -(28) Scan parquet default.web_returns -Output [1]: [wr_order_number#18] +(26) Scan parquet default.web_returns +Output [1]: [wr_order_number#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 16] -Input [1]: [wr_order_number#18] +(27) ColumnarToRow [codegen id : 15] +Input [1]: [wr_order_number#17] -(30) Filter [codegen id : 16] -Input [1]: [wr_order_number#18] -Condition : isnotnull(wr_order_number#18) +(28) Filter [codegen id : 15] +Input [1]: [wr_order_number#17] +Condition : isnotnull(wr_order_number#17) -(31) Exchange -Input [1]: [wr_order_number#18] -Arguments: hashpartitioning(wr_order_number#18, 5), true, [id=#19] +(29) Exchange +Input [1]: [wr_order_number#17] +Arguments: hashpartitioning(wr_order_number#17, 5), true, [id=#18] -(32) Sort [codegen id : 17] -Input [1]: [wr_order_number#18] -Arguments: [wr_order_number#18 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 16] +Input [1]: [wr_order_number#17] +Arguments: [wr_order_number#17 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin [codegen id : 18] +(31) SortMergeJoin [codegen id : 17] Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Right keys [1]: [wr_order_number#17] Join condition: None -(34) Project [codegen id : 18] -Output [1]: [wr_order_number#18] -Input [2]: [ws_order_number#4, wr_order_number#18] +(32) Project [codegen id : 17] +Output [1]: [wr_order_number#17] +Input [2]: [ws_order_number#4, wr_order_number#17] -(35) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Right keys [1]: [wr_order_number#17] Join condition: None -(36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +(34) Scan parquet default.customer_address +Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 19] -Input [2]: [ca_address_sk#20, ca_state#21] +(35) ColumnarToRow [codegen id : 18] +Input [2]: [ca_address_sk#19, ca_state#20] -(38) Filter [codegen id : 19] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) +(36) Filter [codegen id : 18] +Input [2]: [ca_address_sk#19, ca_state#20] +Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = IL)) AND isnotnull(ca_address_sk#19)) -(39) Project [codegen id : 19] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +(37) Project [codegen id : 18] +Output [1]: [ca_address_sk#19] +Input [2]: [ca_address_sk#19, ca_state#20] -(40) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +(38) BroadcastExchange +Input [1]: [ca_address_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(41) BroadcastHashJoin [codegen id : 22] +(39) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#19] Join condition: None -(42) Project [codegen id : 22] +(40) Project [codegen id : 21] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#20] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#19] -(43) Scan parquet default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] +(41) Scan parquet default.web_site +Output [2]: [web_site_sk#22, web_company_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#23, web_company_name#24] +(42) ColumnarToRow [codegen id : 19] +Input [2]: [web_site_sk#22, web_company_name#23] -(45) Filter [codegen id : 20] -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) +(43) Filter [codegen id : 19] +Input [2]: [web_site_sk#22, web_company_name#23] +Condition : ((isnotnull(web_company_name#23) AND (web_company_name#23 = pri)) AND isnotnull(web_site_sk#22)) -(46) Project [codegen id : 20] -Output [1]: [web_site_sk#23] -Input [2]: [web_site_sk#23, web_company_name#24] +(44) Project [codegen id : 19] +Output [1]: [web_site_sk#22] +Input [2]: [web_site_sk#22, web_company_name#23] -(47) BroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +(45) BroadcastExchange +Input [1]: [web_site_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(48) BroadcastHashJoin [codegen id : 22] +(46) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#23] +Right keys [1]: [web_site_sk#22] Join condition: None -(49) Project [codegen id : 22] +(47) Project [codegen id : 21] Output [4]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] +Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#22] -(50) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_date#27] +(48) Scan parquet default.date_dim +Output [2]: [d_date_sk#25, d_date#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#26, d_date#27] +(49) ColumnarToRow [codegen id : 20] +Input [2]: [d_date_sk#25, d_date#26] -(52) Filter [codegen id : 21] -Input [2]: [d_date_sk#26, d_date#27] -Condition : (((isnotnull(d_date#27) AND (d_date#27 >= 10623)) AND (d_date#27 <= 10683)) AND isnotnull(d_date_sk#26)) +(50) Filter [codegen id : 20] +Input [2]: [d_date_sk#25, d_date#26] +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10623)) AND (d_date#26 <= 10683)) AND isnotnull(d_date_sk#25)) -(53) Project [codegen id : 21] -Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_date#27] +(51) Project [codegen id : 20] +Output [1]: [d_date_sk#25] +Input [2]: [d_date_sk#25, d_date#26] -(54) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +(52) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(55) BroadcastHashJoin [codegen id : 22] +(53) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#26] +Right keys [1]: [d_date_sk#25] Join condition: None -(56) Project [codegen id : 22] +(54) Project [codegen id : 21] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#26] +Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#25] -(57) HashAggregate [codegen id : 22] +(55) HashAggregate [codegen id : 21] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#29, sum(UnscaledValue(ws_net_profit#6))#30] -Results [3]: [ws_order_number#4, sum#31, sum#32] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#28, sum(UnscaledValue(ws_net_profit#6))#29] +Results [3]: [ws_order_number#4, sum#30, sum#31] -(58) Exchange -Input [3]: [ws_order_number#4, sum#31, sum#32] -Arguments: hashpartitioning(ws_order_number#4, 5), true, [id=#33] +(56) Exchange +Input [3]: [ws_order_number#4, sum#30, sum#31] +Arguments: hashpartitioning(ws_order_number#4, 5), true, [id=#32] -(59) HashAggregate [codegen id : 23] -Input [3]: [ws_order_number#4, sum#31, sum#32] +(57) HashAggregate [codegen id : 22] +Input [3]: [ws_order_number#4, sum#30, sum#31] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#29, sum(UnscaledValue(ws_net_profit#6))#30] -Results [3]: [ws_order_number#4, sum#31, sum#32] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#28, sum(UnscaledValue(ws_net_profit#6))#29] +Results [3]: [ws_order_number#4, sum#30, sum#31] -(60) HashAggregate [codegen id : 23] -Input [3]: [ws_order_number#4, sum#31, sum#32] +(58) HashAggregate [codegen id : 22] +Input [3]: [ws_order_number#4, sum#30, sum#31] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#29, sum(UnscaledValue(ws_net_profit#6))#30, count(ws_order_number#4)#34] -Results [3]: [sum#31, sum#32, count#35] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#28, sum(UnscaledValue(ws_net_profit#6))#29, count(ws_order_number#4)#33] +Results [3]: [sum#30, sum#31, count#34] -(61) Exchange -Input [3]: [sum#31, sum#32, count#35] -Arguments: SinglePartition, true, [id=#36] +(59) Exchange +Input [3]: [sum#30, sum#31, count#34] +Arguments: SinglePartition, true, [id=#35] -(62) HashAggregate [codegen id : 24] -Input [3]: [sum#31, sum#32, count#35] +(60) HashAggregate [codegen id : 23] +Input [3]: [sum#30, sum#31, count#34] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#29, sum(UnscaledValue(ws_net_profit#6))#30, count(ws_order_number#4)#34] -Results [3]: [count(ws_order_number#4)#34 AS order count #37, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#29,17,2) AS total shipping cost #38, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#30,17,2) AS total net profit #39] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#28, sum(UnscaledValue(ws_net_profit#6))#29, count(ws_order_number#4)#33] +Results [3]: [count(ws_order_number#4)#33 AS order count #36, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#28,17,2) AS total shipping cost #37, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#29,17,2) AS total net profit #38] -(63) TakeOrderedAndProject -Input [3]: [order count #37, total shipping cost #38, total net profit #39] -Arguments: 100, [order count #37 ASC NULLS FIRST], [order count #37, total shipping cost #38, total net profit #39] +(61) TakeOrderedAndProject +Input [3]: [order count #36, total shipping cost #37, total net profit #38] +Arguments: 100, [order count #36 ASC NULLS FIRST], [order count #36, total shipping cost #37, total net profit #38] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index bdcbb87b372dc..da48d34c72a04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] - WholeStageCodegen (24) + WholeStageCodegen (23) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (23) + WholeStageCodegen (22) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] InputAdapter Exchange [ws_order_number] #2 - WholeStageCodegen (22) + WholeStageCodegen (21) HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] BroadcastHashJoin [ws_ship_date_sk,d_date_sk] @@ -18,7 +18,7 @@ TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] InputAdapter SortMergeJoin [ws_order_number,wr_order_number] - WholeStageCodegen (9) + WholeStageCodegen (8) Sort [ws_order_number] InputAdapter Exchange [ws_order_number] #3 @@ -32,78 +32,74 @@ TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - WholeStageCodegen (8) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #5 - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #6 - WholeStageCodegen (3) - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #6 - WholeStageCodegen (18) + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #5 + WholeStageCodegen (3) + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #5 + WholeStageCodegen (17) Project [wr_order_number] SortMergeJoin [ws_order_number,wr_order_number] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (14) Sort [ws_order_number] InputAdapter - Exchange [ws_order_number] #7 - WholeStageCodegen (14) + Exchange [ws_order_number] #6 + WholeStageCodegen (13) Project [ws_order_number] SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (10) Sort [ws_order_number] InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #6 + ReusedExchange [ws_warehouse_sk,ws_order_number] #5 InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [ws_order_number] InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #6 + ReusedExchange [ws_warehouse_sk,ws_order_number] #5 InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (16) Sort [wr_order_number] InputAdapter - Exchange [wr_order_number] #8 - WholeStageCodegen (16) + Exchange [wr_order_number] #7 + WholeStageCodegen (15) Filter [wr_order_number] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_order_number] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (19) + BroadcastExchange #8 + WholeStageCodegen (18) Project [ca_address_sk] Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (20) + BroadcastExchange #9 + WholeStageCodegen (19) Project [web_site_sk] Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_site [web_site_sk,web_company_name] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (21) + BroadcastExchange #10 + WholeStageCodegen (20) Project [d_date_sk] Filter [d_date,d_date_sk] ColumnarToRow diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index be29acb6d3a7c..7db94a702488a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -895,6 +895,170 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { } } + test("SPARK-33399: aliases should be handled properly in PartitioningCollection output" + + " partitioning") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("t1", "t2", "t3") { + spark.range(10).repartition($"id").createTempView("t1") + spark.range(20).repartition($"id").createTempView("t2") + spark.range(30).repartition($"id").createTempView("t3") + val planned = sql( + """ + |SELECT t3.id as t3id + |FROM ( + | SELECT t1.id as t1id, t2.id as t2id + | FROM t1, t2 + | WHERE t1.id = t2.id + |) t12, t3 + |WHERE t1id = t3.id + """.stripMargin).queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.size == 3) + + val projects = planned.collect { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case PartitioningCollection(Seq(HashPartitioning(Seq(k1: AttributeReference), _), + HashPartitioning(Seq(k2: AttributeReference), _))) if k1.name == "t1id" => + true + case _ => false + })) + } + } + } + + test("SPARK-33399: aliases should be handled properly in HashPartitioning") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("t1", "t2", "t3") { + spark.range(10).repartition($"id").createTempView("t1") + spark.range(20).repartition($"id").createTempView("t2") + spark.range(30).repartition($"id").createTempView("t3") + val planned = sql( + """ + |SELECT t1id, t3.id as t3id + |FROM ( + | SELECT t1.id as t1id + | FROM t1 LEFT SEMI JOIN t2 + | ON t1.id = t2.id + |) t12 INNER JOIN t3 + |WHERE t1id = t3.id + """.stripMargin).queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.size == 3) + + val projects = planned.collect { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case HashPartitioning(Seq(a: AttributeReference), _) => a.name == "t1id" + case _ => false + })) + } + } + } + + test("SPARK-33399: alias handling should happen properly for RangePartitioning") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = spark.range(1, 100) + .select(col("id").as("id1")).groupBy("id1").count() + // Plan for this will be Range -> ProjectWithAlias -> HashAggregate -> HashAggregate + // if Project normalizes alias in its Range outputPartitioning, then no Exchange should come + // in between HashAggregates + val planned = df.queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.isEmpty) + + val projects = planned.collect { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case RangePartitioning(Seq(SortOrder(ar: AttributeReference, _, _, _)), _) => + ar.name == "id1" + case _ => false + })) + } + } + + test("SPARK-33399: aliased should be handled properly " + + "for partitioning and sortorder involving complex expressions") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("t1", "t2", "t3") { + spark.range(10).select(col("id").as("id1")).createTempView("t1") + spark.range(20).select(col("id").as("id2")).createTempView("t2") + spark.range(30).select(col("id").as("id3")).createTempView("t3") + val planned = sql( + """ + |SELECT t3.id3 as t3id + |FROM ( + | SELECT t1.id1 as t1id, t2.id2 as t2id + | FROM t1, t2 + | WHERE t1.id1 * 10 = t2.id2 * 10 + |) t12, t3 + |WHERE t1id * 10 = t3.id3 * 10 + """.stripMargin).queryExecution.executedPlan + val sortNodes = planned.collect { case s: SortExec => s } + assert(sortNodes.size == 3) + val exchangeNodes = planned.collect { case e: ShuffleExchangeExec => e } + assert(exchangeNodes.size == 3) + + val projects = planned.collect { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case PartitioningCollection(Seq(HashPartitioning(Seq(Multiply(ar1, _, _)), _), + HashPartitioning(Seq(Multiply(ar2, _, _)), _))) => + Seq(ar1, ar2) match { + case Seq(ar1: AttributeReference, ar2: AttributeReference) => + ar1.name == "t1id" && ar2.name == "id2" + case _ => + false + } + case _ => false + })) + + } + } + } + + test("SPARK-33399: alias handling should happen properly for SinglePartition") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = spark.range(1, 100, 1, 1) + .select(col("id").as("id1")).groupBy("id1").count() + val planned = df.queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.isEmpty) + + val projects = planned.collect { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case SinglePartition => true + case _ => false + })) + } + } + + test("SPARK-33399: No extra exchanges in case of" + + " [Inner Join -> Project with aliases -> HashAggregate]") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("t1", "t2") { + spark.range(10).repartition($"id").createTempView("t1") + spark.range(20).repartition($"id").createTempView("t2") + val planned = sql( + """ + |SELECT t1id, t2id + |FROM ( + | SELECT t1.id as t1id, t2.id as t2id + | FROM t1 INNER JOIN t2 + | WHERE t1.id = t2.id + |) t12 + |GROUP BY t1id, t2id + """.stripMargin).queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.size == 2) + + val projects = planned.collect { case p: ProjectExec => p } + assert(projects.exists(_.outputPartitioning match { + case PartitioningCollection(Seq(HashPartitioning(Seq(k1: AttributeReference), _), + HashPartitioning(Seq(k2: AttributeReference), _))) => + k1.name == "t1id" && k2.name == "t2id" + case _ => false + })) + } + } + } + test("aliases to expressions should not be replaced") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTempView("df1", "df2") {