diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 70112ce3e60bb..1941c9efa9975 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -921,8 +921,18 @@ object InferFiltersFromGenerate extends Rule[LogicalPlan] { } } -abstract class InferFiltersRule extends Rule[LogicalPlan] - with PredicateHelper with ConstraintHelper { +/** + * Generate a list of additional filters from an operator's existing constraint but remove those + * that are either already part of the operator's condition or are part of the operator's child + * constraints. These filters are currently inserted to the existing conditions in the Filter + * operators and on either side of Join operators. + * + * Note: While this optimization is applicable to a lot of types of join, it primarily benefits + * Inner and LeftSemi joins. + */ +object InferFiltersFromConstraints extends Rule[LogicalPlan] + with PredicateHelper with ConstraintHelper { + def apply(plan: LogicalPlan): LogicalPlan = { if (SQLConf.get.constraintPropagationEnabled) { inferFilters(plan) @@ -931,14 +941,6 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] } } - protected def getBaseConstraints( - left: LogicalPlan, - right: LogicalPlan, - conditionOpt: Option[Expression]): ExpressionSet = { - left.constraints.union(right.constraints) - .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) - } - private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform { case filter @ Filter(condition, child) => val newFilters = filter.constraints -- @@ -949,25 +951,25 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] filter } - case join @ Join(left, right, joinType, _, _) => + case join @ Join(left, right, joinType, conditionOpt, _) => joinType match { // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an // inner join, it just drops the right side in the final output. case _: InnerLike | LeftSemi => - val allConstraints = getAllConstraints(join) + val allConstraints = getAllConstraints(left, right, conditionOpt) val newLeft = inferNewFilter(left, allConstraints) val newRight = inferNewFilter(right, allConstraints) join.copy(left = newLeft, right = newRight) // For right outer join, we can only infer additional filters for left side. case RightOuter => - val allConstraints = getAllConstraints(join) + val allConstraints = getAllConstraints(left, right, conditionOpt) val newLeft = inferNewFilter(left, allConstraints) join.copy(left = newLeft) // For left join, we can only infer additional filters for right side. case LeftOuter | LeftAnti => - val allConstraints = getAllConstraints(join) + val allConstraints = getAllConstraints(left, right, conditionOpt) val newRight = inferNewFilter(right, allConstraints) join.copy(right = newRight) @@ -975,7 +977,14 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] } } - protected def getAllConstraints(join: Join): ExpressionSet + private def getAllConstraints( + left: LogicalPlan, + right: LogicalPlan, + conditionOpt: Option[Expression]): ExpressionSet = { + val baseConstraints = left.constraints.union(right.constraints) + .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) + baseConstraints.union(inferAdditionalConstraints(baseConstraints)) + } private def inferNewFilter(plan: LogicalPlan, constraints: ExpressionSet): LogicalPlan = { val newPredicates = constraints @@ -991,22 +1000,6 @@ abstract class InferFiltersRule extends Rule[LogicalPlan] } } -/** - * Generate a list of additional filters from an operator's existing constraint but remove those - * that are either already part of the operator's condition or are part of the operator's child - * constraints. These filters are currently inserted to the existing conditions in the Filter - * operators and on either side of Join operators. - * - * Note: While this optimization is applicable to a lot of types of join, it primarily benefits - * Inner and LeftSemi joins. - */ -object InferFiltersFromConstraints extends InferFiltersRule { - override def getAllConstraints(join: Join): ExpressionSet = { - val baseConstraints = getBaseConstraints(join.left, join.right, join.condition) - baseConstraints.union(inferAdditionalConstraints(baseConstraints)) - } -} - /** * Combines all adjacent [[Union]] operators into a single [[Union]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala index f2ab3e22dcdf2..b92ab41f0169b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/InferDynamicPruningFilters.scala @@ -17,25 +17,79 @@ package org.apache.spark.sql.execution.dynamicpruning -import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ExpressionSet} -import org.apache.spark.sql.catalyst.optimizer.InferFiltersRule -import org.apache.spark.sql.catalyst.plans.logical.Join - -object InferDynamicPruningFilters extends InferFiltersRule { - override def getAllConstraints(join: Join): ExpressionSet = { - val baseConstraints = getBaseConstraints(join.left, join.right, join.condition) - val filtered = inferAdditionalConstraints(baseConstraints, true).filter { +import org.apache.spark.sql.catalyst.expressions.{And, DynamicPruningSubquery, ExpressionSet, PredicateHelper} +import org.apache.spark.sql.catalyst.plans.{InnerLike, LeftAnti, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.catalyst.plans.logical.{ConstraintHelper, Filter, Join, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.dynamicpruning.PartitionPruning._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Similar to InferFiltersFromConstraints, this one only infer DynamicPruning filters. + */ +object InferDynamicPruningFilters extends Rule[LogicalPlan] + with PredicateHelper with ConstraintHelper { + + def apply(plan: LogicalPlan): LogicalPlan = { + if (SQLConf.get.constraintPropagationEnabled) { + inferFilters(plan) + } else { + plan + } + } + + private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform { + case join @ Join(left, right, joinType, _, _) => + joinType match { + // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an + // inner join, it just drops the right side in the final output. + case _: InnerLike | LeftSemi => + val allConstraints = inferDynamicPrunings(join) + val newLeft = inferNewFilter(left, allConstraints) + val newRight = inferNewFilter(right, allConstraints) + join.copy(left = newLeft, right = newRight) + + // For right outer join, we can only infer additional filters for left side. + case RightOuter => + val allConstraints = inferDynamicPrunings(join) + val newLeft = inferNewFilter(left, allConstraints) + join.copy(left = newLeft) + + // For left join, we can only infer additional filters for right side. + case LeftOuter | LeftAnti => + val allConstraints = inferDynamicPrunings(join) + val newRight = inferNewFilter(right, allConstraints) + join.copy(right = newRight) + + case _ => join + } + } + + def inferDynamicPrunings(join: Join): ExpressionSet = { + val baseConstraints = join.left.constraints.union(join.right.constraints) + .union(ExpressionSet(join.condition.map(splitConjunctivePredicates).getOrElse(Nil))) + inferAdditionalConstraints(baseConstraints, true).filter { case DynamicPruningSubquery( pruningKey, buildQuery, buildKeys, broadcastKeyIndex, _, _) => - PartitionPruning.getPartitionTableScan(pruningKey, join) match { + getPartitionTableScan(pruningKey, join) match { case Some(partScan) => - val otherExpr = buildKeys(broadcastKeyIndex) - PartitionPruning.pruningHasBenefit(pruningKey, partScan, otherExpr, buildQuery) + pruningHasBenefit(pruningKey, partScan, buildKeys(broadcastKeyIndex), buildQuery) case _ => false } case _ => false } - baseConstraints.union(filtered) + } + + private def inferNewFilter(plan: LogicalPlan, dynamicPrunings: ExpressionSet): LogicalPlan = { + val newPredicates = dynamicPrunings + .filter { c => + c.references.nonEmpty && c.references.subsetOf(plan.outputSet) && c.deterministic + } -- plan.constraints + if (newPredicates.isEmpty) { + plan + } else { + Filter(newPredicates.reduce(And), plan) + } } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 841c7284d9540..c25973f527384 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -1,67 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * Project (52) - +- * SortMergeJoin Inner (51) - :- * Sort (45) - : +- Exchange (44) - : +- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (36) - : : +- * Filter (35) - : : +- SortMergeJoin ExistenceJoin(exists#1) (34) - : : :- SortMergeJoin ExistenceJoin(exists#2) (26) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (55) ++- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (44) + : +- Exchange (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (35) + : : +- * Filter (34) + : : +- SortMergeJoin ExistenceJoin(exists#1) (33) + : : :- SortMergeJoin ExistenceJoin(exists#2) (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * 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 (25) - : : : +- Exchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (21) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (29) - : +- BroadcastExchange (41) - : +- * Project (40) - : +- * Filter (39) - : +- * ColumnarToRow (38) - : +- Scan parquet default.customer_address (37) - +- * Sort (50) - +- Exchange (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.customer_demographics (46) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- * Filter (38) + : +- * ColumnarToRow (37) + : +- Scan parquet default.customer_address (36) + +- * Sort (49) + +- Exchange (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.customer_demographics (45) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -69,7 +68,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -84,241 +83,236 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#7) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 2002)) AND (d_moy#12 >= 1)) AND (d_moy#12 <= 4)) AND isnotnull(d_date_sk#10)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(22) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(23) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(24) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(27) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(29) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(30) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(31) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(32) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(33) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(35) Filter [codegen id : 13] +(34) Filter [codegen id : 13] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(36) Project [codegen id : 13] +(35) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(37) Scan parquet default.customer_address +(36) Scan parquet default.customer_address Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(38) ColumnarToRow [codegen id : 12] +(37) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#21, ca_county#22] -(39) Filter [codegen id : 12] +(38) Filter [codegen id : 12] Input [2]: [ca_address_sk#21, ca_county#22] Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) -(40) Project [codegen id : 12] +(39) Project [codegen id : 12] Output [1]: [ca_address_sk#21] Input [2]: [ca_address_sk#21, ca_county#22] -(41) BroadcastExchange +(40) BroadcastExchange Input [1]: [ca_address_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(42) BroadcastHashJoin [codegen id : 13] +(41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#21] Join condition: None -(43) Project [codegen id : 13] +(42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] -(44) Exchange +(43) Exchange Input [1]: [c_current_cdemo_sk#4] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] -(45) Sort [codegen id : 14] +(44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(46) Scan parquet default.customer_demographics +(45) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 15] +(46) ColumnarToRow [codegen id : 15] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(48) Filter [codegen id : 15] +(47) Filter [codegen id : 15] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Condition : isnotnull(cd_demo_sk#25) -(49) Exchange +(48) Exchange Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] -(50) Sort [codegen id : 16] +(49) Sort [codegen id : 16] Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 -(51) SortMergeJoin [codegen id : 17] +(50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#25] Join condition: None -(52) Project [codegen id : 17] +(51) Project [codegen id : 17] Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -(53) HashAggregate [codegen id : 17] +(52) HashAggregate [codegen id : 17] Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#35] Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -(54) Exchange +(53) Exchange Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] -(55) HashAggregate [codegen id : 18] +(54) HashAggregate [codegen id : 18] Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#38] Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] -(56) TakeOrderedAndProject +(55) TakeOrderedAndProject Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (57) +ReusedExchange (56) -(57) ReusedExchange [Reuses operator id: 13] +(56) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index fa8f3c241be38..c10844dccad8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -37,12 +37,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 0e011f1c44c0b..07bcd309e9ce6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,59 +1,58 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * Filter (30) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (30) + : : +- * Filter (29) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (18) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.customer_demographics (39) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (41) + +- * Filter (40) + +- * ColumnarToRow (39) + +- Scan parquet default.customer_demographics (38) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -61,224 +60,219 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#6) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(18) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(19) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(20) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(21) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(22) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(23) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(25) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(28) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(30) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_county#21] -(34) Filter [codegen id : 7] +(33) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_county#21] Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) -(35) Project [codegen id : 7] +(34) Project [codegen id : 7] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_county#21] -(36) BroadcastExchange +(35) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(37) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(38) Project [codegen id : 9] +(37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] -(39) Scan parquet default.customer_demographics +(38) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 8] +(39) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(41) Filter [codegen id : 8] +(40) Filter [codegen id : 8] Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#23) -(42) BroadcastExchange +(41) BroadcastExchange Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(43) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(44) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(45) HashAggregate [codegen id : 9] +(44) HashAggregate [codegen id : 9] Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#33] Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -(46) Exchange +(45) Exchange Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] -(47) HashAggregate [codegen id : 10] +(46) HashAggregate [codegen id : 10] Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] -(48) TakeOrderedAndProject +(47) TakeOrderedAndProject Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (49) +ReusedExchange (48) -(49) ReusedExchange [Reuses operator id: 11] +(48) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 78861b51e4f6d..261bb6152b22a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,12 +23,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 05b0c9b7dfff5..d4ab3f60494d9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -162,7 +162,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -170,7 +170,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 341784cbfdeae..4ee1ff31e9beb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 8208efcb1755a..85a3a63413f92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -135,7 +135,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -143,7 +143,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index a66dd29c4927d..f9b76560e363c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 3109147596cf8..76f5ac48f96cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -144,7 +144,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -152,7 +152,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index be55d3b2fecfc..1d5907e319658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -75,7 +75,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 7315c864712eb..01bcb83622844 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -120,7 +120,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -128,7 +128,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 592d91ddf08fd..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -67,7 +67,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 154c74df64b64..9c56438870f64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -1,59 +1,57 @@ == Physical Plan == -* Sort (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- SortMergeJoin LeftAnti (21) - : : : :- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.catalog_sales (7) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.catalog_returns (15) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.customer_address (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.call_center (29) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.date_dim (36) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.catalog_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * Project (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.customer_address (20) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.call_center (27) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.date_dim (34) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -61,7 +59,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) +Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -79,194 +77,184 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(15) Scan parquet default.catalog_returns +(14) Scan parquet default.catalog_returns Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] - -(17) Filter [codegen id : 6] +(15) ColumnarToRow [codegen id : 6] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_order_number#13) -(18) Project [codegen id : 6] +(16) Project [codegen id : 6] Output [1]: [cr_order_number#13] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -(19) Exchange +(17) Exchange Input [1]: [cr_order_number#13] Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(20) Sort [codegen id : 7] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#13] Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin +(19) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#13] Join condition: None -(22) Scan parquet default.customer_address +(20) Scan parquet default.customer_address Output [2]: [ca_address_sk#16, ca_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 8] +(21) ColumnarToRow [codegen id : 8] Input [2]: [ca_address_sk#16, ca_state#17] -(24) Filter [codegen id : 8] +(22) Filter [codegen id : 8] Input [2]: [ca_address_sk#16, ca_state#17] Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) -(25) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [1]: [ca_address_sk#16] Input [2]: [ca_address_sk#16, ca_state#17] -(26) BroadcastExchange +(24) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(27) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join condition: None -(28) Project [codegen id : 11] +(26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -(29) Scan parquet default.call_center +(27) Scan parquet default.call_center Output [2]: [cc_call_center_sk#19, cc_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cc_call_center_sk#19, cc_county#20] -(31) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [2]: [cc_call_center_sk#19, cc_county#20] Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) -(32) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [cc_call_center_sk#19] Input [2]: [cc_call_center_sk#19, cc_county#20] -(33) BroadcastExchange +(31) BroadcastExchange Input [1]: [cc_call_center_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(34) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#19] Join condition: None -(35) Project [codegen id : 11] +(33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] -(36) Scan parquet default.date_dim +(34) Scan parquet default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -(38) Filter [codegen id : 10] +(36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) -(39) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [1]: [d_date_sk#22] Input [2]: [d_date_sk#22, d_date#23] -(40) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#22] Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(44) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(45) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [sum#27, sum#28, count#30] -(46) Exchange +(44) Exchange Input [3]: [sum#27, sum#28, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(47) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] -(48) Sort [codegen id : 12] +(46) Sort [codegen id : 12] Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 5627d2f61e900..393a5baec4400 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,20 +35,18 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - Filter [cs_order_number,cs_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - Filter [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 392a265dcff07..dde6a9f564859 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -1,59 +1,57 @@ == Physical Plan == -* Sort (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- SortMergeJoin LeftAnti (21) - : : : :- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.catalog_sales (7) - : : : +- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.catalog_returns (15) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.date_dim (22) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.customer_address (29) - +- BroadcastExchange (40) - +- * Project (39) - +- * Filter (38) - +- * ColumnarToRow (37) - +- Scan parquet default.call_center (36) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.catalog_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * Project (16) + : : : +- * ColumnarToRow (15) + : : : +- Scan parquet default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.date_dim (20) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- * ColumnarToRow (28) + : +- Scan parquet default.customer_address (27) + +- BroadcastExchange (38) + +- * Project (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.call_center (34) (1) Scan parquet default.catalog_sales Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -61,7 +59,7 @@ Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho (3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) AND isnotnull(cs_order_number#5)) AND isnotnull(cs_warehouse_sk#4)) +Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) (4) Project [codegen id : 1] Output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -79,194 +77,184 @@ Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_order_number), IsNotNull(cs_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -Condition : (isnotnull(cs_order_number#5) AND isnotnull(cs_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#5#11] Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(15) Scan parquet default.catalog_returns +(14) Scan parquet default.catalog_returns Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] - -(17) Filter [codegen id : 6] +(15) ColumnarToRow [codegen id : 6] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_order_number#13) -(18) Project [codegen id : 6] +(16) Project [codegen id : 6] Output [1]: [cr_order_number#13] Input [2]: [cr_order_number#13, cr_returned_date_sk#14] -(19) Exchange +(17) Exchange Input [1]: [cr_order_number#13] Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(20) Sort [codegen id : 7] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#13] Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 -(21) SortMergeJoin +(19) SortMergeJoin Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#13] Join condition: None -(22) Scan parquet default.date_dim +(20) Scan parquet default.date_dim Output [2]: [d_date_sk#16, d_date#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 8] +(21) ColumnarToRow [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -(24) Filter [codegen id : 8] +(22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) -(25) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [1]: [d_date_sk#16] Input [2]: [d_date_sk#16, d_date#17] -(26) BroadcastExchange +(24) BroadcastExchange Input [1]: [d_date_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(27) BroadcastHashJoin [codegen id : 11] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#16] Join condition: None -(28) Project [codegen id : 11] +(26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] -(29) Scan parquet default.customer_address +(27) 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,GA), IsNotNull(ca_address_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(28) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#19, ca_state#20] -(31) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [2]: [ca_address_sk#19, ca_state#20] Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) -(32) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_state#20] -(33) BroadcastExchange +(31) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(34) BroadcastHashJoin [codegen id : 11] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#19] Join condition: None -(35) Project [codegen id : 11] +(33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] -(36) Scan parquet default.call_center +(34) Scan parquet default.call_center Output [2]: [cc_call_center_sk#22, cc_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 10] Input [2]: [cc_call_center_sk#22, cc_county#23] -(38) Filter [codegen id : 10] +(36) Filter [codegen id : 10] Input [2]: [cc_call_center_sk#22, cc_county#23] Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) -(39) Project [codegen id : 10] +(37) Project [codegen id : 10] Output [1]: [cc_call_center_sk#22] Input [2]: [cc_call_center_sk#22, cc_county#23] -(40) BroadcastExchange +(38) BroadcastExchange Input [1]: [cc_call_center_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(41) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#22] Join condition: None -(42) Project [codegen id : 11] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(44) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] Results [3]: [cs_order_number#5, sum#27, sum#28] -(45) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [sum#27, sum#28, count#30] -(46) Exchange +(44) Exchange Input [3]: [sum#27, sum#28, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(47) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] -(48) Sort [codegen id : 12] +(46) Sort [codegen id : 12] Input [3]: [order count #32, total shipping cost #33, total net profit #34] Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 90655232d690e..5edd1426dd8f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (12) Exchange [cs_order_number] #2 WholeStageCodegen (1) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_warehouse_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] @@ -35,20 +35,18 @@ WholeStageCodegen (12) Exchange [cs_order_number] #3 WholeStageCodegen (3) Project [cs_warehouse_sk,cs_order_number] - Filter [cs_order_number,cs_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter Exchange [cr_order_number] #4 WholeStageCodegen (6) Project [cr_order_number] - Filter [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) 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 46acb1a9eebdb..aa96edd4c7fba 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,104 +1,102 @@ == Physical Plan == -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- Union (97) - :- * Project (60) - : +- * BroadcastHashJoin Inner BuildRight (59) - : :- * Project (53) - : : +- SortMergeJoin LeftSemi (52) - : : :- * Sort (33) - : : : +- Exchange (32) - : : : +- * Project (31) - : : : +- SortMergeJoin LeftSemi (30) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- * 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 (51) - : : +- * Project (50) - : : +- * Filter (49) - : : +- * HashAggregate (48) - : : +- * HashAggregate (47) - : : +- * Project (46) - : : +- * SortMergeJoin Inner (45) - : : :- * Sort (39) - : : : +- Exchange (38) - : : : +- * Project (37) - : : : +- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.store_sales (34) - : : +- * 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 (96) - +- * BroadcastHashJoin Inner BuildRight (95) - :- * Project (93) - : +- SortMergeJoin LeftSemi (92) - : :- * Sort (80) - : : +- Exchange (79) - : : +- * Project (78) - : : +- SortMergeJoin LeftSemi (77) - : : :- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.web_sales (61) - : : +- * Sort (76) - : : +- * Project (75) - : : +- * Filter (74) - : : +- * HashAggregate (73) - : : +- * HashAggregate (72) - : : +- * Project (71) - : : +- * SortMergeJoin Inner (70) - : : :- * Sort (67) - : : : +- ReusedExchange (66) - : : +- * Sort (69) - : : +- ReusedExchange (68) - : +- * Sort (91) - : +- * Project (90) - : +- * Filter (89) - : +- * HashAggregate (88) - : +- * HashAggregate (87) - : +- * Project (86) - : +- * SortMergeJoin Inner (85) - : :- * Sort (82) - : : +- ReusedExchange (81) - : +- * Sort (84) - : +- ReusedExchange (83) - +- ReusedExchange (94) +* HashAggregate (98) ++- Exchange (97) + +- * HashAggregate (96) + +- Union (95) + :- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * Project (52) + : : +- SortMergeJoin LeftSemi (51) + : : :- * Sort (32) + : : : +- Exchange (31) + : : : +- * Project (30) + : : : +- SortMergeJoin LeftSemi (29) + : : : :- * Sort (4) + : : : : +- Exchange (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- * Sort (28) + : : : +- * Project (27) + : : : +- * Filter (26) + : : : +- * HashAggregate (25) + : : : +- * HashAggregate (24) + : : : +- * Project (23) + : : : +- * SortMergeJoin Inner (22) + : : : :- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * Filter (7) + : : : : : +- * ColumnarToRow (6) + : : : : : +- Scan parquet default.store_sales (5) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (21) + : : : +- Exchange (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.item (17) + : : +- * Sort (50) + : : +- * Project (49) + : : +- * Filter (48) + : : +- * HashAggregate (47) + : : +- * HashAggregate (46) + : : +- * Project (45) + : : +- * SortMergeJoin Inner (44) + : : :- * Sort (38) + : : : +- Exchange (37) + : : : +- * Project (36) + : : : +- * Filter (35) + : : : +- * ColumnarToRow (34) + : : : +- Scan parquet default.store_sales (33) + : : +- * 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 (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- SortMergeJoin LeftSemi (90) + : :- * Sort (78) + : : +- Exchange (77) + : : +- * Project (76) + : : +- SortMergeJoin LeftSemi (75) + : : :- * Sort (63) + : : : +- Exchange (62) + : : : +- * ColumnarToRow (61) + : : : +- Scan parquet default.web_sales (60) + : : +- * Sort (74) + : : +- * Project (73) + : : +- * Filter (72) + : : +- * HashAggregate (71) + : : +- * HashAggregate (70) + : : +- * Project (69) + : : +- * SortMergeJoin Inner (68) + : : :- * Sort (65) + : : : +- ReusedExchange (64) + : : +- * Sort (67) + : : +- ReusedExchange (66) + : +- * Sort (89) + : +- * Project (88) + : +- * Filter (87) + : +- * HashAggregate (86) + : +- * HashAggregate (85) + : +- * Project (84) + : +- * SortMergeJoin Inner (83) + : :- * Sort (80) + : : +- ReusedExchange (79) + : +- * Sort (82) + : +- ReusedExchange (81) + +- ReusedExchange (92) (1) Scan parquet default.catalog_sales @@ -106,25 +104,20 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 1] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Exchange +(3) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#7] -(5) Sort [codegen id : 2] +(4) Sort [codegen id : 2] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Scan parquet default.store_sales +(5) Scan parquet default.store_sales Output [2]: [ss_item_sk#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -132,422 +125,417 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 4] +(6) ColumnarToRow [codegen id : 4] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] -(8) Filter [codegen id : 4] +(7) Filter [codegen id : 4] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#11, d_date#12, d_year#13] 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 -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#11, d_date#12, d_year#13] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#11, d_date#12, d_year#13] Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [2]: [d_date_sk#11, d_date#12] Input [3]: [d_date_sk#11, d_date#12, d_year#13] -(13) BroadcastExchange +(12) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [2]: [ss_item_sk#8, d_date#12] Input [4]: [ss_item_sk#8, ss_sold_date_sk#9, d_date_sk#11, d_date#12] -(16) Exchange +(15) Exchange Input [2]: [ss_item_sk#8, d_date#12] Arguments: hashpartitioning(ss_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#15] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(18) Scan parquet default.item +(17) Scan parquet default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 6] Input [2]: [i_item_sk#16, i_item_desc#17] -(20) Filter [codegen id : 6] +(19) Filter [codegen id : 6] Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(21) Exchange +(20) Exchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: hashpartitioning(i_item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(22) Sort [codegen id : 7] +(21) Sort [codegen id : 7] Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 -(23) SortMergeJoin [codegen id : 8] +(22) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#16] Join condition: None -(24) Project [codegen id : 8] +(23) Project [codegen id : 8] Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] -(25) HashAggregate [codegen id : 8] +(24) HashAggregate [codegen id : 8] Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#20] Results [4]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12, count#21] -(26) HashAggregate [codegen id : 8] +(25) HashAggregate [codegen id : 8] Input [4]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12, count#21] Keys [3]: [substr(i_item_desc#17, 1, 30)#19, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS count(1)#24] -(27) Filter [codegen id : 8] +(26) Filter [codegen id : 8] Input [2]: [item_sk#23, count(1)#24] Condition : (count(1)#24 > 4) -(28) Project [codegen id : 8] +(27) Project [codegen id : 8] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#24] -(29) Sort [codegen id : 8] +(28) Sort [codegen id : 8] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(30) SortMergeJoin +(29) SortMergeJoin Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#23] Join condition: None -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(32) Exchange +(31) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#25] -(33) Sort [codegen id : 10] +(32) Sort [codegen id : 10] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_sales +(33) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 11] +(34) ColumnarToRow [codegen id : 11] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(36) Filter [codegen id : 11] +(35) Filter [codegen id : 11] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) -(37) Project [codegen id : 11] +(36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(38) Exchange +(37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] -(39) Sort [codegen id : 12] +(38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(40) Scan parquet default.customer +(39) Scan parquet default.customer Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] +(40) ColumnarToRow [codegen id : 13] Input [1]: [c_customer_sk#30] -(42) Filter [codegen id : 13] +(41) Filter [codegen id : 13] Input [1]: [c_customer_sk#30] Condition : isnotnull(c_customer_sk#30) -(43) Exchange +(42) Exchange Input [1]: [c_customer_sk#30] Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] -(44) Sort [codegen id : 14] +(43) Sort [codegen id : 14] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 15] +(44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(46) Project [codegen id : 15] +(45) Project [codegen id : 15] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(47) HashAggregate [codegen id : 15] +(46) HashAggregate [codegen id : 15] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] -(48) HashAggregate [codegen id : 15] +(47) HashAggregate [codegen id : 15] Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(49) Filter [codegen id : 15] +(48) Filter [codegen id : 15] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(50) Project [codegen id : 15] +(49) Project [codegen id : 15] Output [1]: [c_customer_sk#30] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(51) Sort [codegen id : 15] +(50) Sort [codegen id : 15] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin +(51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#30] Join condition: None -(53) Project [codegen id : 17] +(52) Project [codegen id : 17] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(54) Scan parquet default.date_dim +(53) Scan parquet default.date_dim Output [3]: [d_date_sk#11, d_year#13, d_moy#40] 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 : 16] +(54) ColumnarToRow [codegen id : 16] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -(56) Filter [codegen id : 16] +(55) Filter [codegen id : 16] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) -(57) Project [codegen id : 16] +(56) Project [codegen id : 16] Output [1]: [d_date_sk#11] Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -(58) BroadcastExchange +(57) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -(59) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#11] Join condition: None -(60) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] -(61) Scan parquet default.web_sales +(60) Scan parquet default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(62) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] - -(63) Filter [codegen id : 18] +(61) ColumnarToRow [codegen id : 18] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(64) Exchange +(62) Exchange Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] -(65) Sort [codegen id : 19] +(63) Sort [codegen id : 19] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: unknown] +(64) ReusedExchange [Reuses operator id: unknown] Output [2]: [ss_item_sk#8, d_date#12] -(67) Sort [codegen id : 22] +(65) Sort [codegen id : 22] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(68) ReusedExchange [Reuses operator id: 21] +(66) ReusedExchange [Reuses operator id: 20] Output [2]: [i_item_sk#16, i_item_desc#17] -(69) Sort [codegen id : 24] +(67) Sort [codegen id : 24] Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 -(70) SortMergeJoin [codegen id : 25] +(68) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#16] Join condition: None -(71) Project [codegen id : 25] +(69) Project [codegen id : 25] Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] -(72) HashAggregate [codegen id : 25] +(70) HashAggregate [codegen id : 25] Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#50] Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#52] Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] -(74) Filter [codegen id : 25] +(72) Filter [codegen id : 25] Input [2]: [item_sk#23, count(1)#53] Condition : (count(1)#53 > 4) -(75) Project [codegen id : 25] +(73) Project [codegen id : 25] Output [1]: [item_sk#23] Input [2]: [item_sk#23, count(1)#53] -(76) Sort [codegen id : 25] +(74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 -(77) SortMergeJoin +(75) SortMergeJoin Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#23] Join condition: None -(78) Project [codegen id : 26] +(76) Project [codegen id : 26] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(79) Exchange +(77) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] -(80) Sort [codegen id : 27] +(78) Sort [codegen id : 27] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(81) ReusedExchange [Reuses operator id: 38] +(79) ReusedExchange [Reuses operator id: 37] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -(82) Sort [codegen id : 29] +(80) Sort [codegen id : 29] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(83) ReusedExchange [Reuses operator id: 43] +(81) ReusedExchange [Reuses operator id: 42] Output [1]: [c_customer_sk#30] -(84) Sort [codegen id : 31] +(82) Sort [codegen id : 31] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(85) SortMergeJoin [codegen id : 32] +(83) SortMergeJoin [codegen id : 32] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(86) Project [codegen id : 32] +(84) Project [codegen id : 32] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(87) HashAggregate [codegen id : 32] +(85) HashAggregate [codegen id : 32] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#55, isEmpty#56] Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] -(88) HashAggregate [codegen id : 32] +(86) HashAggregate [codegen id : 32] Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -(89) Filter [codegen id : 32] +(87) Filter [codegen id : 32] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(90) Project [codegen id : 32] +(88) Project [codegen id : 32] Output [1]: [c_customer_sk#30] Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -(91) Sort [codegen id : 32] +(89) Sort [codegen id : 32] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(90) SortMergeJoin Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#30] Join condition: None -(93) Project [codegen id : 34] +(91) Project [codegen id : 34] Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(94) ReusedExchange [Reuses operator id: 58] +(92) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#11] -(95) BroadcastHashJoin [codegen id : 34] +(93) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#11] Join condition: None -(96) Project [codegen id : 34] +(94) Project [codegen id : 34] 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#61] Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] -(97) Union +(95) Union -(98) HashAggregate [codegen id : 35] +(96) HashAggregate [codegen id : 35] Input [1]: [sales#42] Keys: [] Functions [1]: [partial_sum(sales#42)] Aggregate Attributes [2]: [sum#62, isEmpty#63] Results [2]: [sum#64, isEmpty#65] -(99) Exchange +(97) Exchange Input [2]: [sum#64, isEmpty#65] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] -(100) HashAggregate [codegen id : 36] +(98) HashAggregate [codegen id : 36] Input [2]: [sum#64, isEmpty#65] Keys: [] Functions [1]: [sum(sales#42)] @@ -557,47 +545,47 @@ Results [1]: [sum(sales#42)#67 AS sum(sales)#68] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (101) +ReusedExchange (99) -(101) ReusedExchange [Reuses operator id: 58] +(99) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#11] -Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -ReusedExchange (102) +Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (100) -(102) ReusedExchange [Reuses operator id: 13] +(100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- * HashAggregate (122) - +- * Project (121) - +- * SortMergeJoin Inner (120) - :- * Sort (114) - : +- Exchange (113) - : +- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * Filter (105) - : : +- * ColumnarToRow (104) - : : +- Scan parquet default.store_sales (103) - : +- BroadcastExchange (110) - : +- * Project (109) - : +- * Filter (108) - : +- * ColumnarToRow (107) - : +- Scan parquet default.date_dim (106) - +- * Sort (119) - +- Exchange (118) - +- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet default.customer (115) - - -(103) Scan parquet default.store_sales +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* 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_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -605,124 +593,124 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 2] +(102) ColumnarToRow [codegen id : 2] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -(105) Filter [codegen id : 2] +(103) Filter [codegen id : 2] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) -(106) Scan parquet default.date_dim +(104) Scan parquet default.date_dim Output [2]: [d_date_sk#11, d_year#13] 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 -(107) ColumnarToRow [codegen id : 1] +(105) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_year#13] -(108) Filter [codegen id : 1] +(106) Filter [codegen id : 1] Input [2]: [d_date_sk#11, d_year#13] Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(109) Project [codegen id : 1] +(107) Project [codegen id : 1] Output [1]: [d_date_sk#11] Input [2]: [d_date_sk#11, d_year#13] -(110) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] -(111) BroadcastHashJoin [codegen id : 2] +(109) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join condition: None -(112) Project [codegen id : 2] +(110) Project [codegen id : 2] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] -(113) Exchange +(111) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] -(114) Sort [codegen id : 3] +(112) Sort [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(115) Scan parquet default.customer +(113) Scan parquet default.customer Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 4] +(114) ColumnarToRow [codegen id : 4] Input [1]: [c_customer_sk#30] -(117) Filter [codegen id : 4] +(115) Filter [codegen id : 4] Input [1]: [c_customer_sk#30] Condition : isnotnull(c_customer_sk#30) -(118) Exchange +(116) Exchange Input [1]: [c_customer_sk#30] Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] -(119) Sort [codegen id : 5] +(117) Sort [codegen id : 5] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(120) SortMergeJoin [codegen id : 6] +(118) SortMergeJoin [codegen id : 6] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#30] Join condition: None -(121) Project [codegen id : 6] +(119) Project [codegen id : 6] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -(122) HashAggregate [codegen id : 6] +(120) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#73, isEmpty#74] Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] -(123) HashAggregate [codegen id : 6] +(121) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] -(124) HashAggregate [codegen id : 6] +(122) HashAggregate [codegen id : 6] Input [1]: [csales#78] Keys: [] Functions [1]: [partial_max(csales#78)] Aggregate Attributes [1]: [max#79] Results [1]: [max#80] -(125) Exchange +(123) Exchange Input [1]: [max#80] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] -(126) HashAggregate [codegen id : 7] +(124) HashAggregate [codegen id : 7] Input [1]: [max#80] Keys: [] Functions [1]: [max(csales#78)] Aggregate Attributes [1]: [max(csales#78)#82] Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] -Subquery:4 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 -ReusedExchange (127) +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 +ReusedExchange (125) -(127) ReusedExchange [Reuses operator id: 110] +(125) ReusedExchange [Reuses operator id: 108] Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 e48dc68bce924..1e3e8aaa3cb96 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 @@ -25,12 +25,11 @@ WholeStageCodegen (36) InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 WholeStageCodegen (8) Sort [item_sk] Project [item_sk] @@ -170,11 +169,10 @@ WholeStageCodegen (36) InputAdapter Exchange [ws_item_sk] #15 WholeStageCodegen (18) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 WholeStageCodegen (25) Sort [item_sk] Project [item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index e3f996ad72e41..692671c9f287d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,80 +1,78 @@ == Physical Plan == -* HashAggregate (76) -+- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (47) - : : +- SortMergeJoin LeftSemi (46) - : : :- * Sort (29) - : : : +- Exchange (28) - : : : +- * Project (27) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.catalog_sales (1) - : : : +- BroadcastExchange (25) - : : : +- * Project (24) - : : : +- * Filter (23) - : : : +- * HashAggregate (22) - : : : +- Exchange (21) - : : : +- * HashAggregate (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (17) - : : : +- * Filter (16) - : : : +- * ColumnarToRow (15) - : : : +- Scan parquet default.item (14) - : : +- * Sort (45) - : : +- * Project (44) - : : +- * Filter (43) - : : +- * HashAggregate (42) - : : +- Exchange (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet default.store_sales (30) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet default.customer (34) - : +- BroadcastExchange (52) - : +- * Project (51) - : +- * Filter (50) - : +- * ColumnarToRow (49) - : +- Scan parquet default.date_dim (48) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (69) - : +- SortMergeJoin LeftSemi (68) - : :- * Sort (62) - : : +- Exchange (61) - : : +- * Project (60) - : : +- * BroadcastHashJoin LeftSemi BuildRight (59) - : : :- * Filter (57) - : : : +- * ColumnarToRow (56) - : : : +- Scan parquet default.web_sales (55) - : : +- ReusedExchange (58) - : +- * Sort (67) - : +- * Project (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- ReusedExchange (63) - +- ReusedExchange (70) +* HashAggregate (74) ++- Exchange (73) + +- * HashAggregate (72) + +- Union (71) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (46) + : : +- SortMergeJoin LeftSemi (45) + : : :- * Sort (28) + : : : +- Exchange (27) + : : : +- * Project (26) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (25) + : : : :- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_sales (1) + : : : +- BroadcastExchange (24) + : : : +- * Project (23) + : : : +- * Filter (22) + : : : +- * HashAggregate (21) + : : : +- Exchange (20) + : : : +- * HashAggregate (19) + : : : +- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * Filter (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- Scan parquet default.store_sales (3) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (16) + : : : +- * Filter (15) + : : : +- * ColumnarToRow (14) + : : : +- Scan parquet default.item (13) + : : +- * Sort (44) + : : +- * Project (43) + : : +- * Filter (42) + : : +- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (32) + : : : +- * Filter (31) + : : : +- * ColumnarToRow (30) + : : : +- Scan parquet default.store_sales (29) + : : +- BroadcastExchange (36) + : : +- * Filter (35) + : : +- * ColumnarToRow (34) + : : +- Scan parquet default.customer (33) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- * Filter (49) + : +- * ColumnarToRow (48) + : +- Scan parquet default.date_dim (47) + +- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (67) + : +- SortMergeJoin LeftSemi (66) + : :- * Sort (60) + : : +- Exchange (59) + : : +- * Project (58) + : : +- * BroadcastHashJoin LeftSemi BuildRight (57) + : : :- * ColumnarToRow (55) + : : : +- Scan parquet default.web_sales (54) + : : +- ReusedExchange (56) + : +- * Sort (65) + : +- * Project (64) + : +- * Filter (63) + : +- * HashAggregate (62) + : +- ReusedExchange (61) + +- ReusedExchange (68) (1) Scan parquet default.catalog_sales @@ -82,17 +80,12 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Scan parquet default.store_sales +(3) Scan parquet default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -100,325 +93,320 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 3] +(4) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) Filter [codegen id : 3] +(5) Filter [codegen id : 3] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#12] 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 -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#12] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#10, d_date#11, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] Input [3]: [d_date_sk#10, d_date#11, d_year#12] -(11) BroadcastExchange +(10) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(12) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(13) Project [codegen id : 3] +(12) Project [codegen id : 3] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -(14) Scan parquet default.item +(13) Scan parquet default.item Output [2]: [i_item_sk#14, i_item_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] +(14) ColumnarToRow [codegen id : 2] Input [2]: [i_item_sk#14, i_item_desc#15] -(16) Filter [codegen id : 2] +(15) Filter [codegen id : 2] Input [2]: [i_item_sk#14, i_item_desc#15] Condition : isnotnull(i_item_sk#14) -(17) BroadcastExchange +(16) BroadcastExchange Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] -(18) BroadcastHashJoin [codegen id : 3] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#14] Join condition: None -(19) Project [codegen id : 3] +(18) Project [codegen id : 3] Output [3]: [d_date#11, i_item_sk#14, i_item_desc#15] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#14, i_item_desc#15] -(20) HashAggregate [codegen id : 3] +(19) HashAggregate [codegen id : 3] Input [3]: [d_date#11, 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)#17, i_item_sk#14, d_date#11] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] Results [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] -(21) Exchange +(20) Exchange Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] Arguments: hashpartitioning(substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, 5), ENSURE_REQUIREMENTS, [id=#20] -(22) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 4] Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11, count#19] Keys [3]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#21] Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS count(1)#23] -(23) Filter [codegen id : 4] +(22) Filter [codegen id : 4] Input [2]: [item_sk#22, count(1)#23] Condition : (count(1)#23 > 4) -(24) Project [codegen id : 4] +(23) Project [codegen id : 4] Output [1]: [item_sk#22] Input [2]: [item_sk#22, count(1)#23] -(25) BroadcastExchange +(24) BroadcastExchange Input [1]: [item_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(26) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#22] Join condition: None -(27) Project [codegen id : 5] +(26) Project [codegen id : 5] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(28) Exchange +(27) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#25] -(29) Sort [codegen id : 6] +(28) Sort [codegen id : 6] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(30) Scan parquet default.store_sales +(29) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 8] +(30) ColumnarToRow [codegen id : 8] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(32) Filter [codegen id : 8] +(31) Filter [codegen id : 8] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) -(33) Project [codegen id : 8] +(32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(34) Scan parquet default.customer +(33) Scan parquet default.customer Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 7] +(34) ColumnarToRow [codegen id : 7] Input [1]: [c_customer_sk#29] -(36) Filter [codegen id : 7] +(35) Filter [codegen id : 7] Input [1]: [c_customer_sk#29] Condition : isnotnull(c_customer_sk#29) -(37) BroadcastExchange +(36) BroadcastExchange Input [1]: [c_customer_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] -(38) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#29] Join condition: None -(39) Project [codegen id : 8] +(38) Project [codegen id : 8] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -(40) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 8] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 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] -(41) Exchange +(40) Exchange Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] -(42) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] 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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(43) Filter [codegen id : 9] +(42) Filter [codegen id : 9] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(44) Project [codegen id : 9] +(43) Project [codegen id : 9] Output [1]: [c_customer_sk#29] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -(45) Sort [codegen id : 9] +(44) Sort [codegen id : 9] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin +(45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(47) Project [codegen id : 11] +(46) Project [codegen id : 11] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(48) Scan parquet default.date_dim +(47) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#12, d_moy#40] 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 -(49) ColumnarToRow [codegen id : 10] +(48) ColumnarToRow [codegen id : 10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -(50) Filter [codegen id : 10] +(49) Filter [codegen id : 10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) -(51) Project [codegen id : 10] +(50) Project [codegen id : 10] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -(52) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] -(53) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#10] Join condition: None -(54) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] -(55) Scan parquet default.web_sales +(54) Scan parquet default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] - -(57) Filter [codegen id : 16] +(55) ColumnarToRow [codegen id : 16] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : (isnotnull(ws_item_sk#43) AND isnotnull(ws_bill_customer_sk#44)) -(58) ReusedExchange [Reuses operator id: unknown] +(56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] -(59) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#22] Join condition: None -(60) Project [codegen id : 16] +(58) Project [codegen id : 16] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(61) Exchange +(59) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] -(62) Sort [codegen id : 17] +(60) Sort [codegen id : 17] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(63) ReusedExchange [Reuses operator id: 41] +(61) ReusedExchange [Reuses operator id: 40] Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] -(64) HashAggregate [codegen id : 20] +(62) HashAggregate [codegen id : 20] Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -(65) Filter [codegen id : 20] +(63) Filter [codegen id : 20] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) -(66) Project [codegen id : 20] +(64) Project [codegen id : 20] Output [1]: [c_customer_sk#29] Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -(67) Sort [codegen id : 20] +(65) Sort [codegen id : 20] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin +(66) SortMergeJoin Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#29] Join condition: None -(69) Project [codegen id : 22] +(67) Project [codegen id : 22] Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(70) ReusedExchange [Reuses operator id: 52] +(68) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#10] -(71) BroadcastHashJoin [codegen id : 22] +(69) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 22] +(70) Project [codegen id : 22] 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#53] Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] -(73) Union +(71) Union -(74) HashAggregate [codegen id : 23] +(72) HashAggregate [codegen id : 23] Input [1]: [sales#42] Keys: [] Functions [1]: [partial_sum(sales#42)] Aggregate Attributes [2]: [sum#54, isEmpty#55] Results [2]: [sum#56, isEmpty#57] -(75) Exchange +(73) Exchange Input [2]: [sum#56, isEmpty#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] -(76) HashAggregate [codegen id : 24] +(74) HashAggregate [codegen id : 24] Input [2]: [sum#56, isEmpty#57] Keys: [] Functions [1]: [sum(sales#42)] @@ -428,45 +416,45 @@ Results [1]: [sum(sales#42)#59 AS sum(sales)#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (77) +ReusedExchange (75) -(77) ReusedExchange [Reuses operator id: 52] +(75) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (78) +Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +ReusedExchange (76) -(78) ReusedExchange [Reuses operator id: 11] +(76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (87) - : +- * BroadcastHashJoin Inner BuildRight (86) - : :- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet default.store_sales (79) - : +- BroadcastExchange (85) - : +- * Filter (84) - : +- * ColumnarToRow (83) - : +- Scan parquet default.customer (82) - +- BroadcastExchange (92) - +- * Project (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.date_dim (88) - - -(79) Scan parquet default.store_sales +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (98) ++- Exchange (97) + +- * HashAggregate (96) + +- * HashAggregate (95) + +- Exchange (94) + +- * HashAggregate (93) + +- * Project (92) + +- * BroadcastHashJoin Inner BuildRight (91) + :- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Filter (79) + : : +- * ColumnarToRow (78) + : : +- Scan parquet default.store_sales (77) + : +- BroadcastExchange (83) + : +- * Filter (82) + : +- * ColumnarToRow (81) + : +- Scan parquet default.customer (80) + +- BroadcastExchange (90) + +- * Project (89) + +- * Filter (88) + +- * ColumnarToRow (87) + +- Scan parquet default.date_dim (86) + + +(77) Scan parquet default.store_sales Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -474,116 +462,116 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 3] +(78) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -(81) Filter [codegen id : 3] +(79) Filter [codegen id : 3] Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) -(82) Scan parquet default.customer +(80) Scan parquet default.customer Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [c_customer_sk#29] -(84) Filter [codegen id : 1] +(82) Filter [codegen id : 1] Input [1]: [c_customer_sk#29] Condition : isnotnull(c_customer_sk#29) -(85) BroadcastExchange +(83) BroadcastExchange Input [1]: [c_customer_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] -(86) BroadcastHashJoin [codegen id : 3] +(84) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#29] Join condition: None -(87) Project [codegen id : 3] +(85) Project [codegen id : 3] Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -(88) Scan parquet default.date_dim +(86) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#12] 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 -(89) ColumnarToRow [codegen id : 2] +(87) ColumnarToRow [codegen id : 2] Input [2]: [d_date_sk#10, d_year#12] -(90) Filter [codegen id : 2] +(88) Filter [codegen id : 2] Input [2]: [d_date_sk#10, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(91) Project [codegen id : 2] +(89) Project [codegen id : 2] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#12] -(92) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] -(93) BroadcastHashJoin [codegen id : 3] +(91) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(94) Project [codegen id : 3] +(92) Project [codegen id : 3] Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] -(95) HashAggregate [codegen id : 3] +(93) HashAggregate [codegen id : 3] Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#64, isEmpty#65] Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] -(96) Exchange +(94) Exchange Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] -(97) HashAggregate [codegen id : 4] +(95) HashAggregate [codegen id : 4] Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] -(98) HashAggregate [codegen id : 4] +(96) HashAggregate [codegen id : 4] Input [1]: [csales#70] Keys: [] Functions [1]: [partial_max(csales#70)] Aggregate Attributes [1]: [max#71] Results [1]: [max#72] -(99) Exchange +(97) Exchange Input [1]: [max#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] -(100) HashAggregate [codegen id : 5] +(98) HashAggregate [codegen id : 5] Input [1]: [max#72] Keys: [] Functions [1]: [max(csales#70)] Aggregate Attributes [1]: [max(csales#70)#74] Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] -Subquery:4 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 -ReusedExchange (101) +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 +ReusedExchange (99) -(101) ReusedExchange [Reuses operator id: 92] +(99) ReusedExchange [Reuses operator id: 90] Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 36bc26bc541bc..87f135b3fff0d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -19,12 +19,11 @@ WholeStageCodegen (24) WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -141,11 +140,10 @@ WholeStageCodegen (24) WholeStageCodegen (16) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [item_sk] #15 WholeStageCodegen (20) 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 0ac39b5242dce..35a42fa256325 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 @@ -143,7 +143,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -151,7 +151,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 1] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) +Condition : isnotnull(cs_bill_customer_sk#1) (4) Exchange Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -516,7 +516,7 @@ Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] @@ -524,7 +524,7 @@ Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price (84) Filter [codegen id : 27] Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Condition : (isnotnull(ws_bill_customer_sk#54) AND isnotnull(ws_item_sk#53)) +Condition : isnotnull(ws_bill_customer_sk#54) (85) Exchange Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] 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 110e3e50157ba..1962a53fa34c2 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 @@ -27,7 +27,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (1) - Filter [cs_bill_customer_sk,cs_item_sk] + Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -210,7 +210,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Exchange [ws_item_sk] #17 WholeStageCodegen (27) - Filter [ws_bill_customer_sk,ws_item_sk] + Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 7b4a41ff0813f..385cf017e58c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -109,7 +109,7 @@ Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] @@ -117,7 +117,7 @@ Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, (3) Filter [codegen id : 5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_bill_customer_sk#1) AND isnotnull(cs_item_sk#2)) +Condition : isnotnull(cs_bill_customer_sk#1) (4) Scan parquet default.store_sales Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] @@ -435,7 +435,7 @@ Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_pric Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] @@ -443,7 +443,7 @@ Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price (73) Filter [codegen id : 22] Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Condition : (isnotnull(ws_bill_customer_sk#55) AND isnotnull(ws_item_sk#54)) +Condition : isnotnull(ws_bill_customer_sk#55) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index e405cfaa0d9bf..a18328434b618 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk,cs_item_sk] + Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] @@ -167,7 +167,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (22) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk,ws_item_sk] + Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index aff4892e0231e..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_manufact_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Electronics )) AND isnotnull(i_manufact_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index 6a4f8b5362c74..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category,i_manufact_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index aff4892e0231e..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_manufact_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_manufact_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics ), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Electronics )) AND isnotnull(i_manufact_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 6a4f8b5362c74..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_manufact_id] - Filter [i_category,i_manufact_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_category,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 82127815c5d69..7098389df6776 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -1,69 +1,68 @@ == Physical Plan == -TakeOrderedAndProject (58) -+- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * SortMergeJoin Inner (53) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (38) - : : +- Exchange (37) - : : +- * Project (36) - : : +- * Filter (35) - : : +- SortMergeJoin ExistenceJoin(exists#1) (34) - : : :- SortMergeJoin ExistenceJoin(exists#2) (26) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (57) ++- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (37) + : : +- Exchange (36) + : : +- * Project (35) + : : +- * Filter (34) + : : +- SortMergeJoin ExistenceJoin(exists#1) (33) + : : :- SortMergeJoin ExistenceJoin(exists#2) (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * 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 (25) - : : : +- Exchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (21) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (29) - : +- * Sort (43) - : +- Exchange (42) - : +- * Filter (41) - : +- * ColumnarToRow (40) - : +- Scan parquet default.customer_address (39) - +- * Sort (52) - +- Exchange (51) - +- * Filter (50) - +- * ColumnarToRow (49) - +- Scan parquet default.customer_demographics (48) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- * Sort (42) + : +- Exchange (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_address (38) + +- * Sort (51) + +- Exchange (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.customer_demographics (47) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +70,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -86,249 +85,244 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#7) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_qoy#12)) AND (d_year#11 = 2002)) AND (d_qoy#12 < 4)) AND isnotnull(d_date_sk#10)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(22) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(23) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(24) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(27) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(29) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(30) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(31) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(32) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(33) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(35) Filter [codegen id : 12] +(34) Filter [codegen id : 12] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(36) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(37) Exchange +(36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] -(38) Sort [codegen id : 13] +(37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer_address +(38) Scan parquet default.customer_address Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 14] +(39) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] -(41) Filter [codegen id : 14] +(40) Filter [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] Condition : isnotnull(ca_address_sk#22) -(42) Exchange +(41) Exchange Input [2]: [ca_address_sk#22, ca_state#23] Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] -(43) Sort [codegen id : 15] +(42) Sort [codegen id : 15] Input [2]: [ca_address_sk#22, ca_state#23] Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 16] +(43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#22] Join condition: None -(45) Project [codegen id : 16] +(44) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#4, ca_state#23] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] -(46) Exchange +(45) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] -(47) Sort [codegen id : 17] +(46) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(48) Scan parquet default.customer_demographics +(47) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(50) Filter [codegen id : 18] +(49) Filter [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#26) -(51) Exchange +(50) Exchange Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(52) Sort [codegen id : 19] +(51) Sort [codegen id : 19] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 20] +(52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#26] Join condition: None -(54) Project [codegen id : 20] +(53) Project [codegen id : 20] Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(55) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 20] Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -(56) Exchange +(55) Exchange Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] -(57) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 21] Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] -(58) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (59) +ReusedExchange (58) -(59) ReusedExchange [Reuses operator id: 13] +(58) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 60624e1380e63..6c399f1bd2197 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -43,12 +43,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 661cac7ce3ef0..4dab13491784a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,58 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * Filter (30) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (30) + : : +- * Filter (29) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (18) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer_demographics (37) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,220 +59,215 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#6) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(18) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(19) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(20) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(21) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(22) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(23) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(25) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(28) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(30) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] -(34) Filter [codegen id : 7] +(33) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 9] +(36) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#21] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(38) Scan parquet default.customer_demographics +(37) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(38) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(40) Filter [codegen id : 8] +(39) Filter [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#23) -(41) BroadcastExchange +(40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(42) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(43) Project [codegen id : 9] +(42) Project [codegen id : 9] Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(44) HashAggregate [codegen id : 9] +(43) HashAggregate [codegen id : 9] Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -(45) Exchange +(44) Exchange Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] -(46) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 10] Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] -(47) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (48) +ReusedExchange (47) -(48) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 4010c2656300c..2614f4f8ae881 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,12 +23,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_empl WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 30ecc62144ddb..a754cef8f2962 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (82) -+- * HashAggregate (81) - +- Exchange (80) - +- * HashAggregate (79) - +- * Expand (78) - +- Union (77) +TakeOrderedAndProject (81) ++- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Expand (77) + +- Union (76) :- * HashAggregate (25) : +- Exchange (24) : +- * HashAggregate (23) @@ -51,36 +51,35 @@ TakeOrderedAndProject (82) : : +- * ColumnarToRow (36) : : +- Scan parquet default.catalog_page (35) : +- ReusedExchange (41) - +- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- Union (64) + +- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- Union (63) : : :- * Project (50) : : : +- * Filter (49) : : : +- * ColumnarToRow (48) : : : +- Scan parquet default.web_sales (47) - : : +- * Project (63) - : : +- * SortMergeJoin Inner (62) - : : :- * Sort (55) - : : : +- Exchange (54) - : : : +- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- * Sort (61) - : : +- Exchange (60) - : : +- * Project (59) - : : +- * Filter (58) - : : +- * ColumnarToRow (57) - : : +- Scan parquet default.web_sales (56) - : +- BroadcastExchange (68) - : +- * Filter (67) - : +- * ColumnarToRow (66) - : +- Scan parquet default.web_site (65) - +- ReusedExchange (71) + : : +- * Project (62) + : : +- * SortMergeJoin Inner (61) + : : :- * Sort (54) + : : : +- Exchange (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet default.web_returns (51) + : : +- * Sort (60) + : : +- Exchange (59) + : : +- * Project (58) + : : +- * Filter (57) + : : +- * ColumnarToRow (56) + : : +- Scan parquet default.web_sales (55) + : +- BroadcastExchange (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.web_site (64) + +- ReusedExchange (70) (1) Scan parquet default.store_sales @@ -319,162 +318,157 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) - -(54) Exchange +(53) Exchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] -(55) Sort [codegen id : 15] +(54) Sort [codegen id : 15] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 -(56) Scan parquet default.web_sales +(55) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 16] +(56) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(58) Filter [codegen id : 16] +(57) Filter [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(59) Project [codegen id : 16] +(58) Project [codegen id : 16] Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(60) Exchange +(59) Exchange Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] -(61) Sort [codegen id : 17] +(60) Sort [codegen id : 17] Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 -(62) SortMergeJoin [codegen id : 18] +(61) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(63) Project [codegen id : 18] +(62) Project [codegen id : 18] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(64) Union +(63) Union -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] Condition : isnotnull(web_site_sk#113) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#113, web_site_id#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] -(69) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#113] Join condition: None -(70) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] -(71) ReusedExchange [Reuses operator id: 20] +(70) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] -(72) BroadcastHashJoin [codegen id : 21] +(71) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None -(73) Project [codegen id : 21] +(72) Project [codegen id : 21] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] -(74) HashAggregate [codegen id : 21] +(73) HashAggregate [codegen id : 21] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Keys [1]: [web_site_id#114] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -(75) Exchange +(74) Exchange Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] -(76) HashAggregate [codegen id : 22] +(75) HashAggregate [codegen id : 22] Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Keys [1]: [web_site_id#114] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] -(77) Union +(76) Union -(78) Expand [codegen id : 23] +(77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] -(79) HashAggregate [codegen id : 23] +(78) HashAggregate [codegen id : 23] Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -(80) Exchange +(79) Exchange Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] -(81) HashAggregate [codegen id : 24] +(80) HashAggregate [codegen id : 24] Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] -(82) TakeOrderedAndProject +(81) TakeOrderedAndProject Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (83) +ReusedExchange (82) -(83) ReusedExchange [Reuses operator id: 20] +(82) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (84) +ReusedExchange (83) -(84) ReusedExchange [Reuses operator id: 20] +(83) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index d3097eff9f8bf..6239f80c2d62d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -114,11 +114,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 81cded23e42db..b3eb1f9592ff3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (79) -+- * HashAggregate (78) - +- Exchange (77) - +- * HashAggregate (76) - +- * Expand (75) - +- Union (74) +TakeOrderedAndProject (78) ++- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * Expand (74) + +- Union (73) :- * HashAggregate (25) : +- Exchange (24) : +- * HashAggregate (23) @@ -51,41 +51,39 @@ TakeOrderedAndProject (79) : +- * Filter (40) : +- * ColumnarToRow (39) : +- Scan parquet default.catalog_page (38) - +- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- Union (61) + +- * HashAggregate (72) + +- Exchange (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (63) + : +- * BroadcastHashJoin Inner BuildRight (62) + : :- Union (60) : : :- * Project (50) : : : +- * Filter (49) : : : +- * ColumnarToRow (48) : : : +- Scan parquet default.web_sales (47) - : : +- * Project (60) - : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : :- BroadcastExchange (54) - : : : +- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet default.web_returns (51) - : : +- * Project (58) - : : +- * Filter (57) - : : +- * ColumnarToRow (56) - : : +- Scan parquet default.web_sales (55) - : +- ReusedExchange (62) - +- BroadcastExchange (68) - +- * Filter (67) - +- * ColumnarToRow (66) - +- Scan parquet default.web_site (65) + : : +- * Project (59) + : : +- * BroadcastHashJoin Inner BuildLeft (58) + : : :- BroadcastExchange (53) + : : : +- * ColumnarToRow (52) + : : : +- Scan parquet default.web_returns (51) + : : +- * Project (57) + : : +- * Filter (56) + : : +- * ColumnarToRow (55) + : : +- Scan parquet default.web_sales (54) + : +- ReusedExchange (61) + +- BroadcastExchange (67) + +- * Filter (66) + +- * ColumnarToRow (65) + +- Scan parquet default.web_site (64) (1) Scan parquet default.store_sales Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), isnotnull(cast(ss_sold_date_sk#4 as bigint)), dynamicpruningexpression(cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] +PartitionFilters: [isnotnull(cast(ss_sold_date_sk#4 as bigint)), dynamicpruningexpression(cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -93,7 +91,7 @@ Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_s (3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#1) AND isnotnull(cast(ss_store_sk#1 as bigint))) +Condition : isnotnull(cast(ss_store_sk#1 as bigint)) (4) Project [codegen id : 1] Output [6]: [cast(ss_store_sk#1 as bigint) AS store_sk#6, cast(ss_sold_date_sk#4 as bigint) AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] @@ -297,7 +295,7 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#8 Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct @@ -317,150 +315,145 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] +(53) BroadcastExchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] -(54) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#103] - -(55) Scan parquet default.web_sales +(54) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(56) ColumnarToRow +(55) ColumnarToRow Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(57) Filter +(56) Filter Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(58) Project +(57) Project Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(59) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(60) Project [codegen id : 15] +(59) Project [codegen id : 15] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(61) Union +(60) Union -(62) ReusedExchange [Reuses operator id: 14] +(61) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] -(63) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None -(64) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 17] +(65) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] -(67) Filter [codegen id : 17] +(66) Filter [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] Condition : isnotnull(web_site_sk#112) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#112, web_site_id#113] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] -(69) BroadcastHashJoin [codegen id : 18] +(68) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#112] Join condition: None -(70) Project [codegen id : 18] +(69) Project [codegen id : 18] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] -(71) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 18] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Keys [1]: [web_site_id#113] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -(72) Exchange +(71) Exchange Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] -(73) HashAggregate [codegen id : 19] +(72) HashAggregate [codegen id : 19] Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Keys [1]: [web_site_id#113] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] -(74) Union +(73) Union -(75) Expand [codegen id : 20] +(74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] -(76) HashAggregate [codegen id : 20] +(75) HashAggregate [codegen id : 20] Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -(77) Exchange +(76) Exchange Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] -(78) HashAggregate [codegen id : 21] +(77) HashAggregate [codegen id : 21] Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] -(79) TakeOrderedAndProject +(78) TakeOrderedAndProject Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (80) +ReusedExchange (79) -(80) ReusedExchange [Reuses operator id: 14] +(79) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (81) +ReusedExchange (80) -(81) ReusedExchange [Reuses operator id: 14] +(80) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 3fd10cf9cb948..50dde50e729d1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -111,11 +111,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 322a882f92c48..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ]), IsNotNull(i_item_id)] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_color#15] -Condition : (i_color#15 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#14)) +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index bfec5dcfdf5d0..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color,i_item_id] + Filter [i_color] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_color] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 322a882f92c48..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ]), IsNotNull(i_item_id)] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_color#15] -Condition : (i_color#15 IN (slate ,blanched ,burnished ) AND isnotnull(i_item_id#14)) +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index bfec5dcfdf5d0..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [total_sales,i_item_id] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_color,i_item_id] + Filter [i_color] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_color] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 5c08ac23ebb55..f6dd5a59de4d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -84,7 +84,7 @@ TakeOrderedAndProject (79) Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -96,406 +96,385 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (6) Filter [codegen id : 2] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (9) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) (10) Project [codegen id : 1] -Output [1]: [d_date#6 AS d_date#6#10] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] (11) BroadcastExchange -Input [1]: [d_date#6#10] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] +Input [1]: [d_date#5#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#10] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#9] Join condition: None (13) Project [codegen id : 2] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (14) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#4] Join condition: None (16) Project [codegen id : 4] Output [2]: [ss_item_sk#1, ss_ext_sales_price#2] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#4] (17) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#13, i_item_id#14] +Input [2]: [i_item_sk#12, i_item_id#13] (19) Filter [codegen id : 3] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Input [2]: [i_item_sk#12, i_item_id#13] +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (20) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +Input [2]: [i_item_sk#12, i_item_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#12] Join condition: None (22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#13, i_item_id#14] +Output [2]: [ss_ext_sales_price#2, i_item_id#13] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#2, i_item_sk#12, i_item_id#13] (23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [ss_ext_sales_price#2, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#13, sum#16] (24) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#13, sum#16] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] (25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#13, sum#16] +Keys [1]: [i_item_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] -Results [2]: [i_item_id#14 AS item_id#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS ss_item_rev#21] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] +Results [2]: [i_item_id#13 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] (26) Filter [codegen id : 15] -Input [2]: [item_id#20, ss_item_rev#21] -Condition : isnotnull(ss_item_rev#21) +Input [2]: [item_id#19, ss_item_rev#20] +Condition : isnotnull(ss_item_rev#20) (27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] (29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : isnotnull(cs_item_sk#22) +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#21) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (33) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (35) Filter [codegen id : 5] -Input [2]: [d_date#6, d_week_seq#7] -Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) (36) Project [codegen id : 5] -Output [1]: [d_date#6 AS d_date#6#26] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#24] +Input [2]: [d_date#5, d_week_seq#6] (37) BroadcastExchange -Input [1]: [d_date#6#26] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] +Input [1]: [d_date#5#24] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#26] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#24] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (40) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#24] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#4] Join condition: None (42) Project [codegen id : 8] -Output [2]: [cs_item_sk#22, cs_ext_sales_price#23] -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#5] +Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [cs_item_sk#21] +Right keys [1]: [i_item_sk#12] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#23, i_item_id#14] -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#22, i_item_id#13] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#23, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [i_item_id#14, sum#30] +Input [2]: [cs_ext_sales_price#22, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#13, sum#28] (47) Exchange -Input [2]: [i_item_id#14, sum#30] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [i_item_id#13, sum#28] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#14, sum#30] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#32] -Results [2]: [i_item_id#14 AS item_id#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#32,17,2) AS cs_item_rev#34] +Input [2]: [i_item_id#13, sum#28] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] (49) Filter [codegen id : 9] -Input [2]: [item_id#33, cs_item_rev#34] -Condition : isnotnull(cs_item_rev#34) +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) (50) BroadcastExchange -Input [2]: [item_id#33, cs_item_rev#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#35] +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#33] -Join condition: ((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#20, ss_item_rev#21, cs_item_rev#34] -Input [4]: [item_id#20, ss_item_rev#21, item_id#33, cs_item_rev#34] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#36) +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (59) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (61) Filter [codegen id : 10] -Input [2]: [d_date#6, d_week_seq#7] -Condition : ((isnotnull(d_week_seq#7) AND (d_week_seq#7 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) AND isnotnull(d_date#6)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) (62) Project [codegen id : 10] -Output [1]: [d_date#6 AS d_date#6#40] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#37] +Input [2]: [d_date#5, d_week_seq#6] (63) BroadcastExchange -Input [1]: [d_date#6#40] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#41] +Input [1]: [d_date#5#37] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#40] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#37] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (66) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#4] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#36, ws_ext_sales_price#37] -Input [4]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, d_date_sk#5] +Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#36] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#12] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#37, i_item_id#14] -Input [4]: [ws_item_sk#36, ws_ext_sales_price#37, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#35, i_item_id#13] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#37, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#43] -Results [2]: [i_item_id#14, sum#44] +Input [2]: [ws_ext_sales_price#35, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#13, sum#41] (73) Exchange -Input [2]: [i_item_id#14, sum#44] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [i_item_id#13, sum#41] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#14, sum#44] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#46] -Results [2]: [i_item_id#14 AS item_id#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#46,17,2) AS ws_item_rev#48] +Input [2]: [i_item_id#13, sum#41] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] (75) Filter [codegen id : 14] -Input [2]: [item_id#47, ws_item_rev#48] -Condition : isnotnull(ws_item_rev#48) +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) (76) BroadcastExchange -Input [2]: [item_id#47, ws_item_rev#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#49] +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#47] -Join condition: ((((((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#20, ss_item_rev#21, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#50, cs_item_rev#34, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#34 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#51, ws_item_rev#48, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#48 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#52, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#53] -Input [5]: [item_id#20, ss_item_rev#21, cs_item_rev#34, item_id#47, ws_item_rev#48] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] (79) TakeOrderedAndProject -Input [8]: [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] -Arguments: 100, [item_id#20 ASC NULLS FIRST, ss_item_rev#21 ASC NULLS FIRST], [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -ReusedExchange (80) +Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] +* Project (83) ++- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet default.date_dim (80) -(80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#5] - -Subquery:2 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* Project (84) -+- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.date_dim (81) - - -(81) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +(80) Scan parquet default.date_dim +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] - -(83) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (isnotnull(d_date#6) AND (d_date#6 = 2000-01-03)) - -(84) Project [codegen id : 1] -Output [1]: [d_week_seq#7] -Input [2]: [d_date#6, d_week_seq#7] - -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#25 -ReusedExchange (85) - - -(85) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#5] - -Subquery:4 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +(81) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] -Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#39 -ReusedExchange (86) +(82) Filter [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +(83) Project [codegen id : 1] +Output [1]: [d_week_seq#6] +Input [2]: [d_date#5, d_week_seq#6] -(86) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#5] +Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] -Subquery:6 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 67f864915caca..5642c7dcbf91c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -18,14 +18,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -33,8 +31,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #3 WholeStageCodegen (1) Project [d_date] - Filter [d_week_seq,d_date] - Subquery #2 + Filter [d_week_seq] + Subquery #1 WholeStageCodegen (1) Project [d_week_seq] Filter [d_date] @@ -68,35 +66,33 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - ReusedExchange [d_date_sk] #7 InputAdapter - BroadcastExchange #8 + BroadcastExchange #7 WholeStageCodegen (6) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (5) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #10 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -107,23 +103,21 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #13 + BroadcastExchange #11 WholeStageCodegen (11) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #14 + BroadcastExchange #12 WholeStageCodegen (10) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 499559c701c71..ab4c4ad4ae65b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -84,7 +84,7 @@ TakeOrderedAndProject (79) Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct @@ -96,406 +96,385 @@ Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] +Input [2]: [i_item_sk#4, i_item_id#5] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) +Input [2]: [i_item_sk#4, i_item_id#5] +Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) (7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] +Input [2]: [i_item_sk#4, i_item_id#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] (8) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] +Right keys [1]: [i_item_sk#4] Join condition: None (9) Project [codegen id : 4] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] +Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5] +Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#4, i_item_id#5] (10) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (12) Filter [codegen id : 3] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (15) Filter [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] -Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) (16) Project [codegen id : 2] -Output [1]: [d_date#9 AS d_date#9#13] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] (17) BroadcastExchange -Input [1]: [d_date#9#13] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +Input [1]: [d_date#8#12] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#13] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#12] Join condition: None (19) Project [codegen id : 3] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (20) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#7] Join condition: None (22) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#6] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#8] +Output [2]: [ss_ext_sales_price#2, i_item_id#5] +Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#5, d_date_sk#7] (23) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#2, i_item_id#6] -Keys [1]: [i_item_id#6] +Input [2]: [ss_ext_sales_price#2, i_item_id#5] +Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#6, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#5, sum#16] (24) Exchange -Input [2]: [i_item_id#6, sum#17] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#5, sum#16] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] (25) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#6, sum#17] -Keys [1]: [i_item_id#6] +Input [2]: [i_item_id#5, sum#16] +Keys [1]: [i_item_id#5] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#19] -Results [2]: [i_item_id#6 AS item_id#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#19,17,2) AS ss_item_rev#21] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#18] +Results [2]: [i_item_id#5 AS item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#18,17,2) AS ss_item_rev#20] (26) Filter [codegen id : 15] -Input [2]: [item_id#20, ss_item_rev#21] -Condition : isnotnull(ss_item_rev#21) +Input [2]: [item_id#19, ss_item_rev#20] +Condition : isnotnull(ss_item_rev#20) (27) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Output [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] (29) Filter [codegen id : 8] -Input [3]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : isnotnull(cs_item_sk#22) +Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#21) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] (31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [cs_item_sk#21] +Right keys [1]: [i_item_sk#4] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_id#6] -Input [5]: [cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_sk#5, i_item_id#6] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (36) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (38) Filter [codegen id : 6] -Input [2]: [d_date#9, d_week_seq#10] -Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) (39) Project [codegen id : 6] -Output [1]: [d_date#9 AS d_date#9#26] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#24] +Input [2]: [d_date#8, d_week_seq#9] (40) BroadcastExchange -Input [1]: [d_date#9#26] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] +Input [1]: [d_date#8#24] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#26] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#24] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (43) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#24] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#7] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#23, i_item_id#6] -Input [4]: [cs_ext_sales_price#23, cs_sold_date_sk#24, i_item_id#6, d_date_sk#8] +Output [2]: [cs_ext_sales_price#22, i_item_id#5] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#23, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [i_item_id#6, sum#30] +Input [2]: [cs_ext_sales_price#22, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#5, sum#28] (47) Exchange -Input [2]: [i_item_id#6, sum#30] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [i_item_id#5, sum#28] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#6, sum#30] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#32] -Results [2]: [i_item_id#6 AS item_id#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#32,17,2) AS cs_item_rev#34] +Input [2]: [i_item_id#5, sum#28] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] (49) Filter [codegen id : 9] -Input [2]: [item_id#33, cs_item_rev#34] -Condition : isnotnull(cs_item_rev#34) +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) (50) BroadcastExchange -Input [2]: [item_id#33, cs_item_rev#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#35] +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#33] -Join condition: ((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#20, ss_item_rev#21, cs_item_rev#34] -Input [4]: [item_id#20, ss_item_rev#21, item_id#33, cs_item_rev#34] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#36) +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#36] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#4] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_id#6] -Input [5]: [ws_item_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_sk#5, i_item_id#6] +Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] +Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (62) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (64) Filter [codegen id : 11] -Input [2]: [d_date#9, d_week_seq#10] -Condition : ((isnotnull(d_week_seq#10) AND (d_week_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date#9)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) (65) Project [codegen id : 11] -Output [1]: [d_date#9 AS d_date#9#40] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#37] +Input [2]: [d_date#8, d_week_seq#9] (66) BroadcastExchange -Input [1]: [d_date#9#40] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#41] +Input [1]: [d_date#8#37] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#40] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#37] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (69) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#7] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#37, i_item_id#6] -Input [4]: [ws_ext_sales_price#37, ws_sold_date_sk#38, i_item_id#6, d_date_sk#8] +Output [2]: [ws_ext_sales_price#35, i_item_id#5] +Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#37, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#43] -Results [2]: [i_item_id#6, sum#44] +Input [2]: [ws_ext_sales_price#35, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#5, sum#41] (73) Exchange -Input [2]: [i_item_id#6, sum#44] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [i_item_id#5, sum#41] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#6, sum#44] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#46] -Results [2]: [i_item_id#6 AS item_id#47, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#46,17,2) AS ws_item_rev#48] +Input [2]: [i_item_id#5, sum#41] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] (75) Filter [codegen id : 14] -Input [2]: [item_id#47, ws_item_rev#48] -Condition : isnotnull(ws_item_rev#48) +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) (76) BroadcastExchange -Input [2]: [item_id#47, ws_item_rev#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#49] +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (77) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#47] -Join condition: ((((((((cast(ss_item_rev#21 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true)) AND (cast(ss_item_rev#21 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(cs_item_rev#34 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#48)), DecimalType(19,3), true))) AND (cast(cs_item_rev#34 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#48)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#21)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#21)), DecimalType(20,3), true))) AND (cast(ws_item_rev#48 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#34)), DecimalType(19,3), true))) AND (cast(ws_item_rev#48 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#34)), DecimalType(20,3), true))) +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#20, ss_item_rev#21, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#50, cs_item_rev#34, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#34 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#51, ws_item_rev#48, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#48 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#52, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#21 as decimal(18,2))) + promote_precision(cast(cs_item_rev#34 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#48 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#53] -Input [5]: [item_id#20, ss_item_rev#21, cs_item_rev#34, item_id#47, ws_item_rev#48] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] (79) TakeOrderedAndProject -Input [8]: [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] -Arguments: 100, [item_id#20 ASC NULLS FIRST, ss_item_rev#21 ASC NULLS FIRST], [item_id#20, ss_item_rev#21, ss_dev#50, cs_item_rev#34, cs_dev#51, ws_item_rev#48, ws_dev#52, average#53] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -ReusedExchange (80) +Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +* Project (83) ++- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet default.date_dim (80) -(80) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#8] - -Subquery:2 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (84) -+- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.date_dim (81) - - -(81) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +(80) Scan parquet default.date_dim +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#9, d_week_seq#10] - -(83) Filter [codegen id : 1] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) - -(84) Project [codegen id : 1] -Output [1]: [d_week_seq#10] -Input [2]: [d_date#9, d_week_seq#10] - -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#25 -ReusedExchange (85) - - -(85) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#8] - -Subquery:4 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +(81) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#8, d_week_seq#9] -Subquery:5 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#39 -ReusedExchange (86) +(82) Filter [codegen id : 1] +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +(83) Project [codegen id : 1] +Output [1]: [d_week_seq#9] +Input [2]: [d_date#8, d_week_seq#9] -(86) ReusedExchange [Reuses operator id: unknown] -Output [1]: [d_date_sk#8] +Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] -Subquery:6 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index ea72681c2f5f1..b2e184c7e279d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -18,21 +18,19 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #2 + BroadcastExchange #3 WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -40,8 +38,8 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev BroadcastExchange #4 WholeStageCodegen (2) Project [d_date] - Filter [d_week_seq,d_date] - Subquery #2 + Filter [d_week_seq] + Subquery #1 WholeStageCodegen (1) Project [d_week_seq] Filter [d_date] @@ -68,35 +66,33 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #8 + BroadcastExchange #7 WholeStageCodegen (7) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (6) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (14) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #10 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -107,25 +103,23 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - ReusedExchange [d_date_sk] #12 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #13 + BroadcastExchange #11 WholeStageCodegen (12) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #14 + BroadcastExchange #12 WholeStageCodegen (11) Project [d_date] - Filter [d_week_seq,d_date] - ReusedSubquery [d_week_seq] #2 + Filter [d_week_seq] + ReusedSubquery [d_week_seq] #1 ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 0085627d7f485..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_category#15] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Music )) AND isnotnull(i_item_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 45041ad094f7b..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category,i_item_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 0085627d7f485..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -149,7 +149,7 @@ Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#10] Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 4] @@ -157,13 +157,13 @@ Input [2]: [i_item_sk#13, i_item_id#14] (20) Filter [codegen id : 4] Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music ), IsNotNull(i_item_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] Input [2]: [i_item_id#14, i_category#15] -Condition : ((isnotnull(i_category#15) AND (i_category#15 = Music )) AND isnotnull(i_item_id#14)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] Output [1]: [i_item_id#14 AS i_item_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 45041ad094f7b..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -45,7 +45,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #5 WholeStageCodegen (4) BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk,i_item_id] + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] @@ -53,7 +53,7 @@ TakeOrderedAndProject [i_item_id,total_sales] BroadcastExchange #6 WholeStageCodegen (3) Project [i_item_id] - Filter [i_category,i_item_id] + Filter [i_category] ColumnarToRow InputAdapter Scan parquet default.item [i_item_id,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 58598de9e10c5..97d7a81baf2a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -1,65 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildLeft (49) - :- BroadcastExchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Project (37) - : : +- SortMergeJoin LeftAnti (36) - : : :- SortMergeJoin LeftAnti (27) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * Project (47) + +- * BroadcastHashJoin Inner BuildLeft (46) + :- BroadcastExchange (42) + : +- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (34) + : : +- SortMergeJoin LeftAnti (33) + : : :- SortMergeJoin LeftAnti (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * 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 (26) - : : : +- Exchange (25) - : : : +- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (22) - : : +- * Sort (35) - : : +- Exchange (34) - : : +- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.catalog_sales (28) - : : +- ReusedExchange (31) - : +- BroadcastExchange (42) - : +- * Project (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_address (38) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.customer_demographics (46) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- * Filter (37) + : +- * ColumnarToRow (36) + : +- Scan parquet default.customer_address (35) + +- * Filter (45) + +- * ColumnarToRow (44) + +- Scan parquet default.customer_demographics (43) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -67,7 +64,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -82,235 +79,220 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#5) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2001)) AND (d_moy#10 >= 4)) AND (d_moy#10 <= 6)) AND isnotnull(d_date_sk#8)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] -Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] - -(21) Filter [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] -Condition : isnotnull(ws_bill_customer_sk#13) -(22) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(23) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(24) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(25) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#13] Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] -(26) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] Arguments: [ws_bill_customer_sk#13 ASC NULLS FIRST], false, 0 -(27) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#13] Join condition: None -(28) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -(30) Filter [codegen id : 10] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_ship_customer_sk#16) - -(31) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(32) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(33) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#16] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(34) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#16] Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] -(35) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#16] Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#16] Join condition: None -(37) Project [codegen id : 13] +(34) Project [codegen id : 13] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(38) Scan parquet default.customer_address +(35) 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: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 12] +(36) ColumnarToRow [codegen id : 12] Input [2]: [ca_address_sk#19, ca_state#20] -(40) Filter [codegen id : 12] +(37) Filter [codegen id : 12] Input [2]: [ca_address_sk#19, ca_state#20] Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) -(41) Project [codegen id : 12] +(38) Project [codegen id : 12] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_state#20] -(42) BroadcastExchange +(39) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(43) BroadcastHashJoin [codegen id : 13] +(40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(44) Project [codegen id : 13] +(41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -(45) BroadcastExchange +(42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(46) Scan parquet default.customer_demographics +(43) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(47) ColumnarToRow +(44) ColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -(48) Filter +(45) Filter Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Condition : isnotnull(cd_demo_sk#23) -(49) BroadcastHashJoin [codegen id : 14] +(46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#23] Join condition: None -(50) Project [codegen id : 14] +(47) Project [codegen id : 14] Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -(51) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -(52) Exchange +(49) Exchange Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] -(53) HashAggregate [codegen id : 15] +(50) HashAggregate [codegen id : 15] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#32] Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] -(54) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (55) +ReusedExchange (52) -(55) ReusedExchange [Reuses operator id: 13] +(52) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index 9a333f419c399..dd7379392ec2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -33,12 +33,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -54,11 +53,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (11) @@ -68,11 +66,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (10) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 5e074df039648..b5307abaf688c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,60 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftAnti BuildRight (31) - : : :- * BroadcastHashJoin LeftAnti BuildRight (23) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin LeftAnti BuildRight (28) + : : :- * BroadcastHashJoin LeftAnti BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- BroadcastExchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet default.catalog_sales (24) - : : +- ReusedExchange (27) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- * Filter (35) - : +- * ColumnarToRow (34) - : +- Scan parquet default.customer_address (33) - +- BroadcastExchange (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet default.customer_demographics (40) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (34) + : +- * Project (33) + : +- * Filter (32) + : +- * ColumnarToRow (31) + : +- Scan parquet default.customer_address (30) + +- BroadcastExchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer_demographics (37) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -62,230 +59,215 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Condition : isnotnull(ss_customer_sk#4) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] - -(18) Filter [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Condition : isnotnull(ws_bill_customer_sk#12) -(19) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(20) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(22) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(23) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#12] Join condition: None -(24) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] - -(26) Filter [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_ship_customer_sk#15) -(27) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(28) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(29) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(30) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] -(31) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#15] Join condition: None -(32) Project [codegen id : 9] +(29) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(33) Scan parquet default.customer_address +(30) Scan parquet default.customer_address Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#18, ca_state#19] -(35) Filter [codegen id : 7] +(32) Filter [codegen id : 7] Input [2]: [ca_address_sk#18, ca_state#19] Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) -(36) Project [codegen id : 7] +(33) Project [codegen id : 7] Output [1]: [ca_address_sk#18] Input [2]: [ca_address_sk#18, ca_state#19] -(37) BroadcastExchange +(34) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] -(38) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join condition: None -(39) Project [codegen id : 9] +(36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(40) Scan parquet default.customer_demographics +(37) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 8] +(38) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -(42) Filter [codegen id : 8] +(39) Filter [codegen id : 8] Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Condition : isnotnull(cd_demo_sk#21) -(43) BroadcastExchange +(40) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(44) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#21] Join condition: None -(45) Project [codegen id : 9] +(42) Project [codegen id : 9] Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -(46) HashAggregate [codegen id : 9] +(43) HashAggregate [codegen id : 9] Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#28] Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -(47) Exchange +(44) Exchange Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] -(48) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 10] Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#31] Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] -(49) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (50) +ReusedExchange (47) -(50) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 7f39a7ff1796f..957b9561b1752 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -22,12 +22,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,11 +40,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -53,11 +51,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 1e8bb4e77fcb4..d31dbc3498ead 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -98,7 +98,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -106,7 +106,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index 8ee6469a58e56..7a5207362a5bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -44,7 +44,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 3f90f0bcea49f..26fec145f4211 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -98,7 +98,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -106,7 +106,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index d7b8e57cc8f4e..f52e7f9a11922 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #4 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -44,7 +44,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index 7d458b55d672d..d006b61d20c33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,73 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (62) -+- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * HashAggregate (30) + : : +- Exchange (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_returns (1) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (18) + : : : +- BroadcastExchange (19) + : : : +- * Project (18) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (17) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (15) - : : : :- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.item (23) - : +- BroadcastExchange (44) - : +- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (34) - : : : +- * ColumnarToRow (33) - : : : +- Scan parquet default.catalog_returns (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- BroadcastExchange (59) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Filter (49) - : : +- * ColumnarToRow (48) - : : +- Scan parquet default.web_returns (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (14) + : : : :- * ColumnarToRow (8) + : : : : +- Scan parquet default.date_dim (7) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * Filter (11) + : : : +- * ColumnarToRow (10) + : : : +- Scan parquet default.date_dim (9) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.item (22) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (33) + : : : +- * ColumnarToRow (32) + : : : +- Scan parquet default.catalog_returns (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -79,295 +78,270 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_date#6] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#5, d_date#6] +Input [2]: [d_date_sk#4, d_date#5] (6) Filter [codegen id : 3] -Input [2]: [d_date_sk#5, d_date#6] -Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_date#6)) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] -(9) Filter [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (isnotnull(d_week_seq#7) AND isnotnull(d_date#6)) - -(10) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +(9) Scan parquet default.date_dim +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] +(10) ColumnarToRow [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] -(12) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (cast(d_date#6 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#7)) +(11) Filter [codegen id : 1] +Input [2]: [d_date#5, d_week_seq#6] +Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(13) Project [codegen id : 1] -Output [1]: [d_week_seq#7 AS d_week_seq#7#8] -Input [2]: [d_date#6, d_week_seq#7] +(12) Project [codegen id : 1] +Output [1]: [d_week_seq#6 AS d_week_seq#6#7] +Input [2]: [d_date#5, d_week_seq#6] -(14) BroadcastExchange -Input [1]: [d_week_seq#7#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] +(13) BroadcastExchange +Input [1]: [d_week_seq#6#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] -(15) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#7] -Right keys [1]: [d_week_seq#7#8] +(14) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#6#7] Join condition: None -(16) Project [codegen id : 2] -Output [1]: [d_date#6 AS d_date#6#10] -Input [2]: [d_date#6, d_week_seq#7] +(15) Project [codegen id : 2] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] -(17) BroadcastExchange -Input [1]: [d_date#6#10] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] +(16) BroadcastExchange +Input [1]: [d_date#5#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] -(18) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#6] -Right keys [1]: [d_date#6#10] +(17) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#9] Join condition: None -(19) Project [codegen id : 3] -Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_date#6] +(18) Project [codegen id : 3] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] -(20) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(19) BroadcastExchange +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(21) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [cast(d_date_sk#5 as bigint)] +Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(22) Project [codegen id : 5] +(21) Project [codegen id : 5] Output [2]: [sr_item_sk#1, sr_return_quantity#2] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#5] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] -(23) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +(22) Scan parquet default.item +Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +(23) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#12, i_item_id#13] -(25) Filter [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +(24) Filter [codegen id : 4] +Input [2]: [i_item_sk#12, i_item_id#13] +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) -(26) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(25) BroadcastExchange +Input [2]: [i_item_sk#12, i_item_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] -(27) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#13 as bigint)] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(28) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#14] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] +(27) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#13] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] -(29) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#14] -Keys [1]: [i_item_id#14] +(28) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#13, sum#16] -(30) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +(29) Exchange +Input [2]: [i_item_id#13, sum#16] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] -(31) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +(30) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#13, sum#16] +Keys [1]: [i_item_id#13] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] -(32) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(31) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(32) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -(34) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +(33) Filter [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) -(35) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] +(34) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#4] -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#5] +(35) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#4] Join condition: None -(37) Project [codegen id : 10] -Output [2]: [cr_item_sk#22, cr_return_quantity#23] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#5] +(36) Project [codegen id : 10] +Output [2]: [cr_item_sk#21, cr_return_quantity#22] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] -(38) ReusedExchange [Reuses operator id: 26] -Output [2]: [i_item_sk#13, i_item_id#14] +(37) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#12, i_item_id#13] -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#13] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#12] Join condition: None -(40) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#14] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#13, i_item_id#14] - -(41) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#14, sum#27] - -(42) Exchange -Input [2]: [i_item_id#14, sum#27] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#28] - -(43) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#14, sum#27] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#29] -Results [2]: [i_item_id#14 AS item_id#30, sum(cr_return_quantity#23)#29 AS cr_item_qty#31] - -(44) BroadcastExchange -Input [2]: [item_id#30, cr_item_qty#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(45) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#30] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#13] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] + +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#13, sum#25] + +(41) Exchange +Input [2]: [i_item_id#13, sum#25] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] + +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#13, sum#25] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] + +(43) BroadcastExchange +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#28] Join condition: None -(46) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#31] -Input [4]: [item_id#20, sr_item_qty#21, item_id#30, cr_item_qty#31] +(45) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(47) Scan parquet default.web_returns -Output [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#35), dynamicpruningexpression(wr_returned_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -(49) Filter [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] -Condition : isnotnull(wr_item_sk#33) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) -(50) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] +(49) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#4] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#35] -Right keys [1]: [cast(d_date_sk#5 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(52) Project [codegen id : 16] -Output [2]: [wr_item_sk#33, wr_return_quantity#34] -Input [4]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35, d_date_sk#5] +(51) Project [codegen id : 16] +Output [2]: [wr_item_sk#31, wr_return_quantity#32] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] -(53) ReusedExchange [Reuses operator id: 26] -Output [2]: [i_item_sk#13, i_item_id#14] +(52) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#12, i_item_id#13] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#33] -Right keys [1]: [cast(i_item_sk#13 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(55) Project [codegen id : 16] -Output [2]: [wr_return_quantity#34, i_item_id#14] -Input [4]: [wr_item_sk#33, wr_return_quantity#34, i_item_sk#13, i_item_id#14] - -(56) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#34, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_item_id#14, sum#37] - -(57) Exchange -Input [2]: [i_item_id#14, sum#37] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#38] - -(58) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#14, sum#37] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum(wr_return_quantity#34)#39] -Results [2]: [i_item_id#14 AS item_id#40, sum(wr_return_quantity#34)#39 AS wr_item_qty#41] - -(59) BroadcastExchange -Input [2]: [item_id#40, wr_item_qty#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] - -(60) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#40] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#32, i_item_id#13] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#32, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#13, sum#35] + +(56) Exchange +Input [2]: [i_item_id#13, sum#35] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#13, sum#35] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] + +(58) BroadcastExchange +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(59) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#38] Join condition: None -(61) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS sr_dev#43, cr_item_qty#31, (((cast(cr_item_qty#31 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS cr_dev#44, wr_item_qty#41, (((cast(wr_item_qty#41 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS wr_dev#45, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#46] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#31, item_id#40, wr_item_qty#41] - -(62) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -ReusedExchange (63) - - -(63) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 -ReusedExchange (64) - - -(64) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#5] - -Subquery:3 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#35 IN dynamicpruning#4 +(60) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(61) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 237c969825d92..b46131afd5ec7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -17,14 +17,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #2 WholeStageCodegen (3) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -33,15 +31,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_date,d_week_seq] + Filter [d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] @@ -68,8 +65,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -90,7 +85,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index f33bf1d745219..c926ab7e3ccb6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (62) -+- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * HashAggregate (30) + : : +- Exchange (29) + : : +- * HashAggregate (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) : : : :- * Filter (3) @@ -18,56 +18,55 @@ TakeOrderedAndProject (62) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin LeftSemi BuildRight (24) + : : +- BroadcastExchange (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * Filter (12) : : : +- * ColumnarToRow (11) : : : +- Scan parquet default.date_dim (10) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin LeftSemi BuildRight (21) - : : :- * Filter (15) - : : : +- * ColumnarToRow (14) - : : : +- Scan parquet default.date_dim (13) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- * ColumnarToRow (17) - : : +- Scan parquet default.date_dim (16) - : +- BroadcastExchange (44) - : +- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (34) - : : : +- * ColumnarToRow (33) - : : : +- Scan parquet default.catalog_returns (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- BroadcastExchange (59) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Filter (49) - : : +- * ColumnarToRow (48) - : : +- Scan parquet default.web_returns (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + : : +- BroadcastExchange (22) + : : +- * Project (21) + : : +- * BroadcastHashJoin LeftSemi BuildRight (20) + : : :- * ColumnarToRow (14) + : : : +- Scan parquet default.date_dim (13) + : : +- BroadcastExchange (19) + : : +- * Project (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.date_dim (15) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (33) + : : : +- * ColumnarToRow (32) + : : : +- Scan parquet default.catalog_returns (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct @@ -79,295 +78,270 @@ Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) (4) Scan parquet default.item -Output [2]: [i_item_sk#5, i_item_id#6] +Output [2]: [i_item_sk#4, i_item_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] +Input [2]: [i_item_sk#4, i_item_id#5] (6) Filter [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) +Input [2]: [i_item_sk#4, i_item_id#5] +Condition : (isnotnull(i_item_sk#4) AND isnotnull(i_item_id#5)) (7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#7] +Input [2]: [i_item_sk#4, i_item_id#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#6] (8) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#5 as bigint)] +Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None (9) Project [codegen id : 5] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] +Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5] +Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#4, i_item_id#5] (10) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#7, d_date#8] (12) Filter [codegen id : 4] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (isnotnull(d_date_sk#8) AND isnotnull(d_date#9)) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] -(15) Filter [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (isnotnull(d_week_seq#10) AND isnotnull(d_date#9)) - -(16) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +(15) Scan parquet default.date_dim +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] +(16) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#8, d_week_seq#9] -(18) Filter [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (cast(d_date#9 as string) IN (2000-06-30,2000-09-27,2000-11-17) AND isnotnull(d_week_seq#10)) +(17) Filter [codegen id : 2] +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(19) Project [codegen id : 2] -Output [1]: [d_week_seq#10 AS d_week_seq#10#11] -Input [2]: [d_date#9, d_week_seq#10] +(18) Project [codegen id : 2] +Output [1]: [d_week_seq#9 AS d_week_seq#9#10] +Input [2]: [d_date#8, d_week_seq#9] -(20) BroadcastExchange -Input [1]: [d_week_seq#10#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(19) BroadcastExchange +Input [1]: [d_week_seq#9#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(21) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#10#11] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_week_seq#9] +Right keys [1]: [d_week_seq#9#10] Join condition: None -(22) Project [codegen id : 3] -Output [1]: [d_date#9 AS d_date#9#13] -Input [2]: [d_date#9, d_week_seq#10] +(21) Project [codegen id : 3] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] -(23) BroadcastExchange -Input [1]: [d_date#9#13] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +(22) BroadcastExchange +Input [1]: [d_date#8#12] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_date#9] -Right keys [1]: [d_date#9#13] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#12] Join condition: None -(25) Project [codegen id : 4] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +(24) Project [codegen id : 4] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] -(26) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +(25) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(27) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [cast(d_date_sk#8 as bigint)] +Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(28) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#6] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#8] +(27) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#5] +Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk#7] -(29) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#6] -Keys [1]: [i_item_id#6] +(28) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#5] +Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#6, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#5, sum#16] -(30) Exchange -Input [2]: [i_item_id#6, sum#17] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#18] +(29) Exchange +Input [2]: [i_item_id#5, sum#16] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] -(31) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#6, sum#17] -Keys [1]: [i_item_id#6] +(30) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#5, sum#16] +Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#6 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] -(32) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(31) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +(32) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -(34) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +(33) Filter [codegen id : 10] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) -(35) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +(34) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#4, i_item_id#5] -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#5] +(35) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#4] Join condition: None -(37) Project [codegen id : 10] -Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#6] -Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#5, i_item_id#6] +(36) Project [codegen id : 10] +Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] +Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] -(38) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] +(37) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#7] -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#8] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#7] Join condition: None -(40) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#6] -Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#6, d_date_sk#8] - -(41) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#6, sum#27] - -(42) Exchange -Input [2]: [i_item_id#6, sum#27] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#28] - -(43) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#6, sum#27] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#29] -Results [2]: [i_item_id#6 AS item_id#30, sum(cr_return_quantity#23)#29 AS cr_item_qty#31] - -(44) BroadcastExchange -Input [2]: [item_id#30, cr_item_qty#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(45) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#30] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#5] +Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] + +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#5, sum#25] + +(41) Exchange +Input [2]: [i_item_id#5, sum#25] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] + +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#5, sum#25] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] + +(43) BroadcastExchange +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#28] Join condition: None -(46) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#31] -Input [4]: [item_id#20, sr_item_qty#21, item_id#30, cr_item_qty#31] +(45) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(47) Scan parquet default.web_returns -Output [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#35), dynamicpruningexpression(wr_returned_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -(49) Filter [codegen id : 16] -Input [3]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35] -Condition : isnotnull(wr_item_sk#33) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) -(50) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_item_id#6] +(49) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#4, i_item_id#5] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#33] -Right keys [1]: [cast(i_item_sk#5 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None -(52) Project [codegen id : 16] -Output [3]: [wr_return_quantity#34, wr_returned_date_sk#35, i_item_id#6] -Input [5]: [wr_item_sk#33, wr_return_quantity#34, wr_returned_date_sk#35, i_item_sk#5, i_item_id#6] +(51) Project [codegen id : 16] +Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] +Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] -(53) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] +(52) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#7] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#35] -Right keys [1]: [cast(d_date_sk#8 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(55) Project [codegen id : 16] -Output [2]: [wr_return_quantity#34, i_item_id#6] -Input [4]: [wr_return_quantity#34, wr_returned_date_sk#35, i_item_id#6, d_date_sk#8] - -(56) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#34, i_item_id#6] -Keys [1]: [i_item_id#6] -Functions [1]: [partial_sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum#36] -Results [2]: [i_item_id#6, sum#37] - -(57) Exchange -Input [2]: [i_item_id#6, sum#37] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [id=#38] - -(58) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#6, sum#37] -Keys [1]: [i_item_id#6] -Functions [1]: [sum(wr_return_quantity#34)] -Aggregate Attributes [1]: [sum(wr_return_quantity#34)#39] -Results [2]: [i_item_id#6 AS item_id#40, sum(wr_return_quantity#34)#39 AS wr_item_qty#41] - -(59) BroadcastExchange -Input [2]: [item_id#40, wr_item_qty#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] - -(60) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#40] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#32, i_item_id#5] +Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#32, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#5, sum#35] + +(56) Exchange +Input [2]: [i_item_id#5, sum#35] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#5, sum#35] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] + +(58) BroadcastExchange +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(59) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#38] Join condition: None -(61) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS sr_dev#43, cr_item_qty#31, (((cast(cr_item_qty#31 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS cr_dev#44, wr_item_qty#41, (((cast(wr_item_qty#41 as double) / cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as double)) / 3.0) * 100.0) AS wr_dev#45, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#31) + wr_item_qty#41) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#46] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#31, item_id#40, wr_item_qty#41] - -(62) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#43, cr_item_qty#31, cr_dev#44, wr_item_qty#41, wr_dev#45, average#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -ReusedExchange (63) - - -(63) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 -ReusedExchange (64) - - -(64) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#8] - -Subquery:3 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#35 IN dynamicpruning#4 +(60) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(61) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index fc2b465906e4e..7fc930f2885fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -17,21 +17,19 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #3 + BroadcastExchange #2 WholeStageCodegen (1) Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #2 + BroadcastExchange #3 WholeStageCodegen (4) Project [d_date_sk] BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk,d_date] + Filter [d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_date] @@ -40,15 +38,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (3) Project [d_date] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Project [d_week_seq] - Filter [d_date,d_week_seq] + Filter [d_date] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] @@ -68,12 +65,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 WholeStageCodegen (17) @@ -90,8 +85,7 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #3 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 19cbce69c0768..2466a4d38f21e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * SortMergeJoin Inner (20) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * SortMergeJoin Inner (19) :- * Sort (13) : +- Exchange (12) : +- * Project (11) @@ -18,12 +18,11 @@ TakeOrderedAndProject (25) : +- * Filter (7) : +- * ColumnarToRow (6) : +- Scan parquet default.reason (5) - +- * Sort (19) - +- Exchange (18) - +- * Project (17) - +- * Filter (16) - +- * ColumnarToRow (15) - +- Scan parquet default.store_sales (14) + +- * Sort (18) + +- Exchange (17) + +- * Project (16) + +- * ColumnarToRow (15) + +- Scan parquet default.store_sales (14) (1) Scan parquet default.store_returns @@ -87,56 +86,51 @@ Arguments: [sr_item_sk#1 ASC NULLS FIRST, sr_ticket_number#3 ASC NULLS FIRST], f Output [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -(16) Filter [codegen id : 4] -Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_ticket_number#12)) - -(17) Project [codegen id : 4] +(16) Project [codegen id : 4] Output [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Input [6]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14, ss_sold_date_sk#15] -(18) Exchange +(17) Exchange Input [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Arguments: hashpartitioning(cast(ss_item_sk#10 as bigint), cast(ss_ticket_number#12 as bigint), 5), ENSURE_REQUIREMENTS, [id=#16] -(19) Sort [codegen id : 5] +(18) Sort [codegen id : 5] Input [5]: [ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] Arguments: [cast(ss_item_sk#10 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#12 as bigint) ASC NULLS FIRST], false, 0 -(20) SortMergeJoin [codegen id : 6] +(19) SortMergeJoin [codegen id : 6] Left keys [2]: [sr_item_sk#1, sr_ticket_number#3] Right keys [2]: [cast(ss_item_sk#10 as bigint), cast(ss_ticket_number#12 as bigint)] Join condition: None -(21) Project [codegen id : 6] +(20) Project [codegen id : 6] Output [2]: [ss_customer_sk#11, CASE WHEN isnotnull(sr_return_quantity#4) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#13 - sr_return_quantity#4) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#14 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#13 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#14 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#17] Input [8]: [sr_item_sk#1, sr_ticket_number#3, sr_return_quantity#4, ss_item_sk#10, ss_customer_sk#11, ss_ticket_number#12, ss_quantity#13, ss_sales_price#14] -(22) HashAggregate [codegen id : 6] +(21) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#11, act_sales#17] Keys [1]: [ss_customer_sk#11] Functions [1]: [partial_sum(act_sales#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [ss_customer_sk#11, sum#20, isEmpty#21] -(23) Exchange +(22) Exchange Input [3]: [ss_customer_sk#11, sum#20, isEmpty#21] Arguments: hashpartitioning(ss_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#22] -(24) HashAggregate [codegen id : 7] +(23) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#11, sum#20, isEmpty#21] Keys [1]: [ss_customer_sk#11] Functions [1]: [sum(act_sales#17)] Aggregate Attributes [1]: [sum(act_sales#17)#23] Results [2]: [ss_customer_sk#11, sum(act_sales#17)#23 AS sumsales#24] -(25) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [2]: [ss_customer_sk#11, sumsales#24] Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt index fcebeb98855be..17464e295b1dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt @@ -35,7 +35,6 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #4 WholeStageCodegen (4) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - Filter [ss_item_sk,ss_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index cd69276ebcb4a..1f3470198cd20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,142 +1,136 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (14) - : +- * SortMergeJoin Inner (13) - : :- * Sort (6) - : : +- Exchange (5) - : : +- * Project (4) - : : +- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- * Sort (12) - : +- Exchange (11) - : +- * Project (10) - : +- * Filter (9) - : +- * ColumnarToRow (8) - : +- Scan parquet default.store_returns (7) - +- BroadcastExchange (19) - +- * Project (18) - +- * Filter (17) - +- * ColumnarToRow (16) - +- Scan parquet default.reason (15) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * BroadcastHashJoin Inner BuildRight (19) + :- * Project (13) + : +- * SortMergeJoin Inner (12) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Project (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (11) + : +- Exchange (10) + : +- * Project (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_returns (6) + +- BroadcastExchange (18) + +- * Project (17) + +- * Filter (16) + +- * ColumnarToRow (15) + +- Scan parquet default.reason (14) (1) Scan parquet default.store_sales Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 1] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#3)) - -(4) Project [codegen id : 1] +(3) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -(5) Exchange +(4) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint), 5), ENSURE_REQUIREMENTS, [id=#7] -(6) Sort [codegen id : 2] +(5) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#3 as bigint) ASC NULLS FIRST], false, 0 -(7) Scan parquet default.store_returns +(6) Scan parquet default.store_returns Output [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 3] +(7) ColumnarToRow [codegen id : 3] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(9) Filter [codegen id : 3] +(8) Filter [codegen id : 3] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_item_sk#8) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_reason_sk#9)) -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Input [5]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(11) Exchange +(10) Exchange Input [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Arguments: hashpartitioning(sr_item_sk#8, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [id=#13] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [4]: [sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] Arguments: [sr_item_sk#8 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin [codegen id : 6] +(12) SortMergeJoin [codegen id : 6] Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#3 as bigint)] Right keys [2]: [sr_item_sk#8, sr_ticket_number#10] Join condition: None -(14) Project [codegen id : 6] +(13) Project [codegen id : 6] Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#9, sr_return_quantity#11] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#8, sr_reason_sk#9, sr_ticket_number#10, sr_return_quantity#11] -(15) Scan parquet default.reason +(14) Scan parquet default.reason Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(16) ColumnarToRow [codegen id : 5] +(15) ColumnarToRow [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -(17) Filter [codegen id : 5] +(16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) -(18) Project [codegen id : 5] +(17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] Input [2]: [r_reason_sk#14, r_reason_desc#15] -(19) BroadcastExchange +(18) BroadcastExchange Input [1]: [r_reason_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(20) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#9] Right keys [1]: [cast(r_reason_sk#14 as bigint)] Join condition: None -(21) Project [codegen id : 6] +(20) Project [codegen id : 6] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#11) THEN CheckOverflow((promote_precision(cast(cast((ss_quantity#4 - sr_return_quantity#11) as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) ELSE CheckOverflow((promote_precision(cast(cast(ss_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2), true) END AS act_sales#17] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#9, sr_return_quantity#11, r_reason_sk#14] -(22) HashAggregate [codegen id : 6] +(21) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#2, act_sales#17] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [3]: [ss_customer_sk#2, sum#20, isEmpty#21] -(23) Exchange +(22) Exchange Input [3]: [ss_customer_sk#2, sum#20, isEmpty#21] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#22] -(24) HashAggregate [codegen id : 7] +(23) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#2, sum#20, isEmpty#21] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#17)] Aggregate Attributes [1]: [sum(act_sales#17)#23] Results [2]: [ss_customer_sk#2, sum(act_sales#17)#23 AS sumsales#24] -(25) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#24] Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index c6441ac0fc899..0d9aec90a2da4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -16,10 +16,9 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] Exchange [ss_item_sk,ss_ticket_number] #2 WholeStageCodegen (1) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - Filter [ss_item_sk,ss_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index eb516affccfed..c77590bf71044 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -1,62 +1,60 @@ == Physical Plan == -* Sort (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- SortMergeJoin LeftAnti (23) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Sort (15) + : : : : +- Exchange (14) + : : : : +- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- * Sort (22) - : : : +- Exchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.web_returns (17) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.customer_address (24) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.web_site (31) - +- BroadcastExchange (42) - +- * Project (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.date_dim (38) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.web_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.web_returns (16) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.customer_address (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.web_site (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.date_dim (36) (1) Scan parquet default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -64,7 +62,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -82,206 +80,196 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(15) Exchange +(14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] -(16) Sort [codegen id : 6] +(15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(17) Scan parquet default.web_returns +(16) Scan parquet default.web_returns Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] - -(19) Filter [codegen id : 7] +(17) ColumnarToRow [codegen id : 7] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -Condition : isnotnull(wr_order_number#14) -(20) Project [codegen id : 7] +(18) Project [codegen id : 7] Output [1]: [wr_order_number#14] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -(21) Exchange +(19) Exchange Input [1]: [wr_order_number#14] Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(22) Sort [codegen id : 8] +(20) Sort [codegen id : 8] Input [1]: [wr_order_number#14] Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 -(23) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#14] Join condition: None -(24) Scan parquet default.customer_address +(22) Scan parquet default.customer_address Output [2]: [ca_address_sk#17, ca_state#18] 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 -(25) ColumnarToRow [codegen id : 9] +(23) ColumnarToRow [codegen id : 9] Input [2]: [ca_address_sk#17, ca_state#18] -(26) Filter [codegen id : 9] +(24) Filter [codegen id : 9] Input [2]: [ca_address_sk#17, ca_state#18] Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) -(27) Project [codegen id : 9] +(25) Project [codegen id : 9] Output [1]: [ca_address_sk#17] Input [2]: [ca_address_sk#17, ca_state#18] -(28) BroadcastExchange +(26) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 12] +(27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#17] Join condition: None -(30) Project [codegen id : 12] +(28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] -(31) Scan parquet default.web_site +(29) Scan parquet default.web_site Output [2]: [web_site_sk#20, web_company_name#21] 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 -(32) ColumnarToRow [codegen id : 10] +(30) ColumnarToRow [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -(33) Filter [codegen id : 10] +(31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) -(34) Project [codegen id : 10] +(32) Project [codegen id : 10] Output [1]: [web_site_sk#20] Input [2]: [web_site_sk#20, web_company_name#21] -(35) BroadcastExchange +(33) BroadcastExchange Input [1]: [web_site_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#20] Join condition: None -(37) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] -(38) Scan parquet default.date_dim +(36) Scan parquet default.date_dim Output [2]: [d_date_sk#23, d_date#24] 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 -(39) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -(40) Filter [codegen id : 11] +(38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) -(41) Project [codegen id : 11] +(39) Project [codegen id : 11] Output [1]: [d_date_sk#23] Input [2]: [d_date_sk#23, d_date#24] -(42) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(43) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#23] Join condition: None -(44) Project [codegen id : 12] +(42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] -(45) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(46) Exchange +(44) Exchange Input [3]: [ws_order_number#5, sum#28, sum#29] Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] -(47) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(48) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [sum#28, sum#29, count#32] -(49) Exchange +(47) Exchange Input [3]: [sum#28, sum#29, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(50) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] -(51) Sort [codegen id : 14] +(49) Sort [codegen id : 14] Input [3]: [order count #34, total shipping cost #35, total net profit #36] Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 6a7459768a8fd..3ef7977c4bc64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -32,7 +32,7 @@ WholeStageCodegen (14) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -42,20 +42,18 @@ WholeStageCodegen (14) Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index e6001eae60e57..9558a01423452 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -1,62 +1,60 @@ == Physical Plan == -* Sort (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- SortMergeJoin LeftAnti (23) - : : : :- * Sort (16) - : : : : +- Exchange (15) - : : : : +- * Project (14) - : : : : +- SortMergeJoin LeftSemi (13) +* Sort (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- SortMergeJoin LeftAnti (21) + : : : :- * Sort (15) + : : : : +- Exchange (14) + : : : : +- * Project (13) + : : : : +- SortMergeJoin LeftSemi (12) : : : : :- * Sort (6) : : : : : +- Exchange (5) : : : : : +- * Project (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.web_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.web_sales (7) - : : : +- * Sort (22) - : : : +- Exchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.web_returns (17) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.date_dim (24) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_address (31) - +- BroadcastExchange (42) - +- * Project (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.web_site (38) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * Project (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.web_sales (7) + : : : +- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.web_returns (16) + : : +- BroadcastExchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.date_dim (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (40) + +- * Project (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.web_site (36) (1) Scan parquet default.web_sales Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -64,7 +62,7 @@ Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (3) Filter [codegen id : 1] Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#5)) AND isnotnull(ws_warehouse_sk#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -82,206 +80,196 @@ Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -Condition : (isnotnull(ws_order_number#5) AND isnotnull(ws_warehouse_sk#4)) - -(10) Project [codegen id : 3] +(9) Project [codegen id : 3] Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] -(11) Exchange +(10) Exchange Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] -(12) Sort [codegen id : 4] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 -(13) SortMergeJoin +(12) SortMergeJoin Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#5#11] Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) -(14) Project [codegen id : 5] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(15) Exchange +(14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] -(16) Sort [codegen id : 6] +(15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 -(17) Scan parquet default.web_returns +(16) Scan parquet default.web_returns Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] - -(19) Filter [codegen id : 7] +(17) ColumnarToRow [codegen id : 7] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -Condition : isnotnull(wr_order_number#14) -(20) Project [codegen id : 7] +(18) Project [codegen id : 7] Output [1]: [wr_order_number#14] Input [2]: [wr_order_number#14, wr_returned_date_sk#15] -(21) Exchange +(19) Exchange Input [1]: [wr_order_number#14] Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] -(22) Sort [codegen id : 8] +(20) Sort [codegen id : 8] Input [1]: [wr_order_number#14] Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 -(23) SortMergeJoin +(21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] Right keys [1]: [wr_order_number#14] Join condition: None -(24) Scan parquet default.date_dim +(22) Scan parquet default.date_dim Output [2]: [d_date_sk#17, d_date#18] 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 -(25) ColumnarToRow [codegen id : 9] +(23) ColumnarToRow [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -(26) Filter [codegen id : 9] +(24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) -(27) Project [codegen id : 9] +(25) Project [codegen id : 9] Output [1]: [d_date_sk#17] Input [2]: [d_date_sk#17, d_date#18] -(28) BroadcastExchange +(26) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 12] +(27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#17] Join condition: None -(30) Project [codegen id : 12] +(28) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] -(31) Scan parquet default.customer_address +(29) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] 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 -(32) ColumnarToRow [codegen id : 10] +(30) ColumnarToRow [codegen id : 10] Input [2]: [ca_address_sk#20, ca_state#21] -(33) Filter [codegen id : 10] +(31) Filter [codegen id : 10] 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)) -(34) Project [codegen id : 10] +(32) Project [codegen id : 10] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_state#21] -(35) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] -(38) Scan parquet default.web_site +(36) Scan parquet default.web_site Output [2]: [web_site_sk#23, web_company_name#24] 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 -(39) ColumnarToRow [codegen id : 11] +(37) ColumnarToRow [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -(40) Filter [codegen id : 11] +(38) Filter [codegen id : 11] 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)) -(41) Project [codegen id : 11] +(39) Project [codegen id : 11] Output [1]: [web_site_sk#23] Input [2]: [web_site_sk#23, web_company_name#24] -(42) BroadcastExchange +(40) BroadcastExchange Input [1]: [web_site_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] -(43) BroadcastHashJoin [codegen id : 12] +(41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#23] Join condition: None -(44) Project [codegen id : 12] +(42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] -(45) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(46) Exchange +(44) Exchange Input [3]: [ws_order_number#5, sum#28, sum#29] Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] -(47) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(48) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [sum#28, sum#29, count#32] -(49) Exchange +(47) Exchange Input [3]: [sum#28, sum#29, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(50) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] -(51) Sort [codegen id : 14] +(49) Sort [codegen id : 14] Input [3]: [order count #34, total shipping cost #35, total net profit #36] Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 1282ca0d0f702..6bcd2aa24ccbb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -32,7 +32,7 @@ WholeStageCodegen (14) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_warehouse_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] @@ -42,20 +42,18 @@ WholeStageCodegen (14) Exchange [ws_order_number] #5 WholeStageCodegen (3) Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter Exchange [wr_order_number] #6 WholeStageCodegen (7) Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) 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 d111152b95384..320a93e19bb27 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 @@ -69,7 +69,7 @@ Output [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, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -77,7 +77,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] 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, ws_sold_date_sk#7] -Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [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] 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 53b0ccfc3e218..439f3935c65a3 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 @@ -29,7 +29,7 @@ WholeStageCodegen (23) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] 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,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index f57897991b8fa..2f719a4be1630 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -69,7 +69,7 @@ Output [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, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_order_number)] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -77,7 +77,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num (3) Filter [codegen id : 1] 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, ws_sold_date_sk#7] -Condition : (((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) AND isnotnull(ws_order_number#4)) +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) (4) Project [codegen id : 1] Output [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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 16bb4ac84f00c..adf95b02f07e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (23) Exchange [ws_order_number] #4 WholeStageCodegen (1) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] 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,ws_sold_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 422a89af349bc..58d28b28dae10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -1,63 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin Inner BuildLeft (47) - :- BroadcastExchange (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- SortMergeJoin LeftSemi (34) - : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildLeft (44) + :- BroadcastExchange (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (32) + : : +- SortMergeJoin LeftSemi (31) + : : :- SortMergeJoin LeftSemi (17) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- * 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 (33) - : : +- Exchange (32) - : : +- Union (31) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (22) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet default.catalog_sales (25) - : : +- ReusedExchange (28) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.customer_address (36) - +- * Filter (46) - +- * ColumnarToRow (45) - +- Scan parquet default.customer_demographics (44) + : : : +- * Sort (16) + : : : +- Exchange (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * ColumnarToRow (7) + : : : : +- Scan parquet default.store_sales (6) + : : : +- BroadcastExchange (12) + : : : +- * Project (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet default.date_dim (8) + : : +- * Sort (30) + : : +- Exchange (29) + : : +- Union (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet default.customer_address (33) + +- * Filter (43) + +- * ColumnarToRow (42) + +- Scan parquet default.customer_demographics (41) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -65,7 +62,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -80,224 +77,209 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#5) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 4)) AND (d_moy#10 <= 7)) AND isnotnull(d_date_sk#8)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] -Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] - -(21) Filter [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] -Condition : isnotnull(ws_bill_customer_sk#13) -(22) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(23) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(24) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(25) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 9] +(24) ColumnarToRow [codegen id : 9] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -(27) Filter [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_ship_customer_sk#16) - -(28) ReusedExchange [Reuses operator id: 13] +(25) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(29) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(30) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(31) Union +(28) Union -(32) Exchange +(29) Exchange Input [1]: [customer_sk#15] Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(33) Sort [codegen id : 10] +(30) Sort [codegen id : 10] Input [1]: [customer_sk#15] Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(31) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#15] Join condition: None -(35) Project [codegen id : 12] +(32) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(36) Scan parquet default.customer_address +(33) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 11] +(34) ColumnarToRow [codegen id : 11] Input [2]: [ca_address_sk#20, ca_county#21] -(38) Filter [codegen id : 11] +(35) Filter [codegen id : 11] Input [2]: [ca_address_sk#20, ca_county#21] Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(39) Project [codegen id : 11] +(36) Project [codegen id : 11] Output [1]: [ca_address_sk#20] Input [2]: [ca_address_sk#20, ca_county#21] -(40) BroadcastExchange +(37) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(41) BroadcastHashJoin [codegen id : 12] +(38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#20] Join condition: None -(42) Project [codegen id : 12] +(39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -(43) BroadcastExchange +(40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(44) Scan parquet default.customer_demographics +(41) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(45) ColumnarToRow +(42) ColumnarToRow Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(46) Filter +(43) Filter Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Condition : isnotnull(cd_demo_sk#24) -(47) BroadcastHashJoin [codegen id : 13] +(44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#24] Join condition: None -(48) Project [codegen id : 13] +(45) Project [codegen id : 13] Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -(49) HashAggregate [codegen id : 13] +(46) HashAggregate [codegen id : 13] Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#33] Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -(50) Exchange +(47) Exchange Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] -(51) HashAggregate [codegen id : 14] +(48) HashAggregate [codegen id : 14] Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#36] Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] -(52) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (53) +ReusedExchange (50) -(53) ReusedExchange [Reuses operator id: 13] +(50) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 1d4fd5a709520..220059a6f5e27 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -32,12 +32,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -54,21 +53,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 1a258296802b5..eac6acc3543ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,59 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * BroadcastHashJoin LeftSemi BuildRight (27) + : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (29) - : : +- Union (28) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (42) - +- * Filter (41) - +- * ColumnarToRow (40) - +- Scan parquet default.customer_demographics (39) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * ColumnarToRow (5) + : : : : +- Scan parquet default.store_sales (4) + : : : +- BroadcastExchange (10) + : : : +- * Project (9) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.date_dim (6) + : : +- BroadcastExchange (26) + : : +- Union (25) + : : :- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * ColumnarToRow (21) + : : : +- Scan parquet default.catalog_sales (20) + : : +- ReusedExchange (22) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (39) + +- * Filter (38) + +- * ColumnarToRow (37) + +- Scan parquet default.customer_demographics (36) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -61,223 +58,208 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Condition : isnotnull(ss_customer_sk#4) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] - -(18) Filter [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Condition : isnotnull(ws_bill_customer_sk#12) -(19) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(20) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(22) Scan parquet default.catalog_sales +(20) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] - -(24) Filter [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_ship_customer_sk#15) -(25) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(26) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(27) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(28) Union +(25) Union -(29) BroadcastExchange +(26) BroadcastExchange Input [1]: [customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(30) BroadcastHashJoin [codegen id : 9] +(27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#14] Join condition: None -(31) Project [codegen id : 9] +(28) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(32) Scan parquet default.customer_address +(29) Scan parquet default.customer_address Output [2]: [ca_address_sk#19, ca_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#19, ca_county#20] -(34) Filter [codegen id : 7] +(31) Filter [codegen id : 7] Input [2]: [ca_address_sk#19, ca_county#20] Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) -(35) Project [codegen id : 7] +(32) Project [codegen id : 7] Output [1]: [ca_address_sk#19] Input [2]: [ca_address_sk#19, ca_county#20] -(36) BroadcastExchange +(33) BroadcastExchange Input [1]: [ca_address_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -(37) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(38) Project [codegen id : 9] +(35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -(39) Scan parquet default.customer_demographics +(36) Scan parquet default.customer_demographics Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 8] +(37) ColumnarToRow [codegen id : 8] Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(41) Filter [codegen id : 8] +(38) Filter [codegen id : 8] Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Condition : isnotnull(cd_demo_sk#22) -(42) BroadcastExchange +(39) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(43) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#22] Join condition: None -(44) Project [codegen id : 9] +(41) Project [codegen id : 9] Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(45) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 9] Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#32] Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -(46) Exchange +(43) Exchange Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] -(47) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 10] Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#35] Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] -(48) TakeOrderedAndProject +(45) TakeOrderedAndProject Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (49) +ReusedExchange (46) -(49) ReusedExchange [Reuses operator id: 11] +(46) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 72526cd0b007a..5077b91e30c1b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,12 +21,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,21 +40,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index f100c3e7e6ebf..312aba94bd9c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -144,7 +144,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -152,7 +152,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index be55d3b2fecfc..1d5907e319658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -75,7 +75,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index b4ab1eb60e90f..258224e574cfb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -120,7 +120,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -128,7 +128,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 592d91ddf08fd..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -67,7 +67,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index b8f60c510269a..9b47400d3b780 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -241,7 +241,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 20] @@ -249,7 +249,7 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index d418fe48a4d67..d27eb5a32b387 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -97,7 +97,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index c7b4a326a053f..04fe14cab2c9b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -214,7 +214,7 @@ Condition : isnotnull(ss_item_sk#1) Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -222,7 +222,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_item_sk#6)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 80c58cd303dc5..1634c1e247a12 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -89,7 +89,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (11) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - Filter [i_brand_id,i_class_id,i_category_id,i_item_sk] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 9657cdfd07984..0a861f4ce8e6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -1,69 +1,68 @@ == Physical Plan == -TakeOrderedAndProject (58) -+- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * SortMergeJoin Inner (53) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (38) - : : +- Exchange (37) - : : +- * Project (36) - : : +- * Filter (35) - : : +- SortMergeJoin ExistenceJoin(exists#1) (34) - : : :- SortMergeJoin ExistenceJoin(exists#2) (26) - : : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (57) ++- * HashAggregate (56) + +- Exchange (55) + +- * HashAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * SortMergeJoin Inner (43) + : :- * Sort (37) + : : +- Exchange (36) + : : +- * Project (35) + : : +- * Filter (34) + : : +- SortMergeJoin ExistenceJoin(exists#1) (33) + : : :- SortMergeJoin ExistenceJoin(exists#2) (25) + : : : :- SortMergeJoin LeftSemi (17) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- * 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 (25) - : : : +- Exchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (21) - : : +- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- Scan parquet default.catalog_sales (27) - : : +- ReusedExchange (29) - : +- * Sort (43) - : +- Exchange (42) - : +- * Filter (41) - : +- * ColumnarToRow (40) - : +- Scan parquet default.customer_address (39) - +- * Sort (52) - +- Exchange (51) - +- * Filter (50) - +- * ColumnarToRow (49) - +- Scan parquet default.customer_demographics (48) + : : : : +- * Sort (16) + : : : : +- Exchange (15) + : : : : +- * Project (14) + : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : :- * ColumnarToRow (7) + : : : : : +- Scan parquet default.store_sales (6) + : : : : +- BroadcastExchange (12) + : : : : +- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet default.date_dim (8) + : : : +- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * ColumnarToRow (27) + : : : +- Scan parquet default.catalog_sales (26) + : : +- ReusedExchange (28) + : +- * Sort (42) + : +- Exchange (41) + : +- * Filter (40) + : +- * ColumnarToRow (39) + : +- Scan parquet default.customer_address (38) + +- * Sort (51) + +- Exchange (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.customer_demographics (47) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -71,7 +70,7 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Exchange Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] @@ -86,249 +85,244 @@ Output [2]: [ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#7) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_qoy#12)) AND (d_year#11 = 2002)) AND (d_qoy#12 < 4)) AND isnotnull(d_date_sk#10)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#7] Input [3]: [ss_customer_sk#7, ss_sold_date_sk#8, d_date_sk#10] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#7] Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#14] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#7] Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#7] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#16), dynamicpruningexpression(ws_sold_date_sk#16 IN dynamicpruning#9)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] -(21) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(22) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(23) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] -(24) Exchange +(23) Exchange Input [1]: [ws_bill_customer_sk#15] Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] -(25) Sort [codegen id : 8] +(24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] Arguments: [ws_bill_customer_sk#15 ASC NULLS FIRST], false, 0 -(26) SortMergeJoin +(25) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#15] Join condition: None -(27) Scan parquet default.catalog_sales +(26) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 10] +(27) ColumnarToRow [codegen id : 10] Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -(29) ReusedExchange [Reuses operator id: 13] +(28) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -(30) BroadcastHashJoin [codegen id : 10] +(29) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#19] Right keys [1]: [d_date_sk#10] Join condition: None -(31) Project [codegen id : 10] +(30) Project [codegen id : 10] Output [1]: [cs_ship_customer_sk#18] Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] -(32) Exchange +(31) Exchange Input [1]: [cs_ship_customer_sk#18] Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] -(33) Sort [codegen id : 11] +(32) Sort [codegen id : 11] Input [1]: [cs_ship_customer_sk#18] Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#18] Join condition: None -(35) Filter [codegen id : 12] +(34) Filter [codegen id : 12] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(36) Project [codegen id : 12] +(35) Project [codegen id : 12] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(37) Exchange +(36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] -(38) Sort [codegen id : 13] +(37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer_address +(38) Scan parquet default.customer_address Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 14] +(39) ColumnarToRow [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] -(41) Filter [codegen id : 14] +(40) Filter [codegen id : 14] Input [2]: [ca_address_sk#22, ca_state#23] Condition : isnotnull(ca_address_sk#22) -(42) Exchange +(41) Exchange Input [2]: [ca_address_sk#22, ca_state#23] Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] -(43) Sort [codegen id : 15] +(42) Sort [codegen id : 15] Input [2]: [ca_address_sk#22, ca_state#23] Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 16] +(43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#22] Join condition: None -(45) Project [codegen id : 16] +(44) Project [codegen id : 16] Output [2]: [c_current_cdemo_sk#4, ca_state#23] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] -(46) Exchange +(45) Exchange Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] -(47) Sort [codegen id : 17] +(46) Sort [codegen id : 17] Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 -(48) Scan parquet default.customer_demographics +(47) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 18] +(48) ColumnarToRow [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(50) Filter [codegen id : 18] +(49) Filter [codegen id : 18] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Condition : isnotnull(cd_demo_sk#26) -(51) Exchange +(50) Exchange Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(52) Sort [codegen id : 19] +(51) Sort [codegen id : 19] Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 20] +(52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#26] Join condition: None -(54) Project [codegen id : 20] +(53) Project [codegen id : 20] Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -(55) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 20] Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -(56) Exchange +(55) Exchange Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] -(57) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 21] Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] -(58) TakeOrderedAndProject +(57) TakeOrderedAndProject Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (59) +ReusedExchange (58) -(59) ReusedExchange [Reuses operator id: 13] +(58) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index 7be34612be79f..ee441a8332a2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -43,12 +43,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index c803c1989723d..12c8230c7e3bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,58 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * Filter (30) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (29) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (30) + : : +- * Filter (29) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (28) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (21) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (14) - : : : : +- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.store_sales (4) - : : : : +- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.date_dim (7) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (18) - : : +- BroadcastExchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * ColumnarToRow (24) - : : : +- Scan parquet default.catalog_sales (23) - : : +- ReusedExchange (25) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : : +- BroadcastExchange (13) + : : : : +- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * ColumnarToRow (5) + : : : : : +- Scan parquet default.store_sales (4) + : : : : +- BroadcastExchange (10) + : : : : +- * Project (9) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.date_dim (6) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_address (31) + +- BroadcastExchange (40) + +- * Filter (39) + +- * ColumnarToRow (38) + +- Scan parquet default.customer_demographics (37) (1) Scan parquet default.customer Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,220 +59,215 @@ Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : ((isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) AND isnotnull(c_customer_sk#3)) +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#6) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#8)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] -(18) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(19) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] Right keys [1]: [d_date_sk#9] Join condition: None -(20) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] -(21) BroadcastExchange +(20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(22) BroadcastHashJoin [codegen id : 9] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#14] Join condition: None -(23) Scan parquet default.catalog_sales +(22) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(23) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -(25) ReusedExchange [Reuses operator id: 11] +(24) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -(26) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#9] Join condition: None -(27) Project [codegen id : 6] +(26) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#17] Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] -(28) BroadcastExchange +(27) BroadcastExchange Input [1]: [cs_ship_customer_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#17] Join condition: None -(30) Filter [codegen id : 9] +(29) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(32) Scan parquet default.customer_address +(31) Scan parquet default.customer_address Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(32) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] -(34) Filter [codegen id : 7] +(33) Filter [codegen id : 7] Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(36) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join condition: None -(37) Project [codegen id : 9] +(36) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, ca_state#21] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(38) Scan parquet default.customer_demographics +(37) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(38) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(40) Filter [codegen id : 8] +(39) Filter [codegen id : 8] Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Condition : isnotnull(cd_demo_sk#23) -(41) BroadcastExchange +(40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(42) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join condition: None -(43) Project [codegen id : 9] +(42) Project [codegen id : 9] Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(44) HashAggregate [codegen id : 9] +(43) HashAggregate [codegen id : 9] Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -(45) Exchange +(44) Exchange Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] -(46) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 10] Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] -(47) TakeOrderedAndProject +(46) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -ReusedExchange (48) +ReusedExchange (47) -(48) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#9] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index c2dd037ca1b47..c6c9e1da58c26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -23,12 +23,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index bae142df5e6d6..1c1b8ad01abfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -1,68 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (57) -+- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * Project (53) - +- * SortMergeJoin Inner (52) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- SortMergeJoin LeftSemi (34) - : : :- SortMergeJoin LeftSemi (18) +TakeOrderedAndProject (54) ++- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (43) + : +- Exchange (42) + : +- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (34) + : : +- Exchange (33) + : : +- * Project (32) + : : +- SortMergeJoin LeftSemi (31) + : : :- SortMergeJoin LeftSemi (17) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- * 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 (33) - : : +- Exchange (32) - : : +- Union (31) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet default.web_sales (19) - : : : +- ReusedExchange (22) - : : +- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet default.catalog_sales (25) - : : +- ReusedExchange (28) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_address (38) - +- * Sort (51) - +- Exchange (50) - +- * Filter (49) - +- * ColumnarToRow (48) - +- Scan parquet default.customer_demographics (47) + : : : +- * Sort (16) + : : : +- Exchange (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * ColumnarToRow (7) + : : : : +- Scan parquet default.store_sales (6) + : : : +- BroadcastExchange (12) + : : : +- * Project (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet default.date_dim (8) + : : +- * Sort (30) + : : +- Exchange (29) + : : +- Union (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- Scan parquet default.web_sales (18) + : : : +- ReusedExchange (20) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * ColumnarToRow (24) + : : : +- Scan parquet default.catalog_sales (23) + : : +- ReusedExchange (25) + : +- * Sort (39) + : +- Exchange (38) + : +- * Filter (37) + : +- * ColumnarToRow (36) + : +- Scan parquet default.customer_address (35) + +- * Sort (48) + +- Exchange (47) + +- * Filter (46) + +- * ColumnarToRow (45) + +- Scan parquet default.customer_demographics (44) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -70,7 +67,7 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 1] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Exchange Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -85,244 +82,229 @@ Output [2]: [ss_customer_sk#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (7) ColumnarToRow [codegen id : 4] Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -(8) Filter [codegen id : 4] -Input [2]: [ss_customer_sk#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#5) - -(9) Scan parquet default.date_dim +(8) Scan parquet default.date_dim Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 3] +(9) ColumnarToRow [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -(11) Filter [codegen id : 3] +(10) Filter [codegen id : 3] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] Condition : ((((isnotnull(d_year#9) AND isnotnull(d_qoy#10)) AND (d_year#9 = 1999)) AND (d_qoy#10 < 4)) AND isnotnull(d_date_sk#8)) -(12) Project [codegen id : 3] +(11) Project [codegen id : 3] Output [1]: [d_date_sk#8] Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -(13) BroadcastExchange +(12) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(14) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join condition: None -(15) Project [codegen id : 4] +(14) Project [codegen id : 4] Output [1]: [ss_customer_sk#5] Input [3]: [ss_customer_sk#5, ss_sold_date_sk#6, d_date_sk#8] -(16) Exchange +(15) Exchange Input [1]: [ss_customer_sk#5] Arguments: hashpartitioning(ss_customer_sk#5, 5), ENSURE_REQUIREMENTS, [id=#12] -(17) Sort [codegen id : 5] +(16) Sort [codegen id : 5] Input [1]: [ss_customer_sk#5] Arguments: [ss_customer_sk#5 ASC NULLS FIRST], false, 0 -(18) SortMergeJoin +(17) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#5] Join condition: None -(19) Scan parquet default.web_sales +(18) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#14), dynamicpruningexpression(ws_sold_date_sk#14 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 7] -Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] - -(21) Filter [codegen id : 7] +(19) ColumnarToRow [codegen id : 7] Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] -Condition : isnotnull(ws_bill_customer_sk#13) -(22) ReusedExchange [Reuses operator id: 13] +(20) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(23) BroadcastHashJoin [codegen id : 7] +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] Right keys [1]: [d_date_sk#8] Join condition: None -(24) Project [codegen id : 7] +(22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13 AS customsk#15] Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] -(25) Scan parquet default.catalog_sales +(23) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] - -(27) Filter [codegen id : 9] +(24) ColumnarToRow [codegen id : 9] Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_ship_customer_sk#16) -(28) ReusedExchange [Reuses operator id: 13] +(25) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -(29) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#8] Join condition: None -(30) Project [codegen id : 9] +(27) Project [codegen id : 9] Output [1]: [cs_ship_customer_sk#16 AS customsk#18] Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] -(31) Union +(28) Union -(32) Exchange +(29) Exchange Input [1]: [customsk#15] Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(33) Sort [codegen id : 10] +(30) Sort [codegen id : 10] Input [1]: [customsk#15] Arguments: [customsk#15 ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(31) SortMergeJoin Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#15] Join condition: None -(35) Project [codegen id : 11] +(32) Project [codegen id : 11] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(36) Exchange +(33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] -(37) Sort [codegen id : 12] +(34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_address +(35) Scan parquet default.customer_address Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 13] +(36) ColumnarToRow [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] -(40) Filter [codegen id : 13] +(37) Filter [codegen id : 13] Input [2]: [ca_address_sk#21, ca_state#22] Condition : isnotnull(ca_address_sk#21) -(41) Exchange +(38) Exchange Input [2]: [ca_address_sk#21, ca_state#22] Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] -(42) Sort [codegen id : 14] +(39) Sort [codegen id : 14] Input [2]: [ca_address_sk#21, ca_state#22] Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 15] +(40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#21] Join condition: None -(44) Project [codegen id : 15] +(41) Project [codegen id : 15] Output [2]: [c_current_cdemo_sk#2, ca_state#22] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] -(45) Exchange +(42) Exchange Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] -(46) Sort [codegen id : 16] +(43) Sort [codegen id : 16] Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 -(47) Scan parquet default.customer_demographics +(44) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 17] +(45) ColumnarToRow [codegen id : 17] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(49) Filter [codegen id : 17] +(46) Filter [codegen id : 17] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Condition : isnotnull(cd_demo_sk#25) -(50) Exchange +(47) Exchange Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] -(51) Sort [codegen id : 18] +(48) Sort [codegen id : 18] Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 -(52) SortMergeJoin [codegen id : 19] +(49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#25] Join condition: None -(53) Project [codegen id : 19] +(50) Project [codegen id : 19] Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(54) HashAggregate [codegen id : 19] +(51) HashAggregate [codegen id : 19] Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -(55) Exchange +(52) Exchange Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] -(56) HashAggregate [codegen id : 20] +(53) HashAggregate [codegen id : 20] Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] -(57) TakeOrderedAndProject +(54) TakeOrderedAndProject Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -ReusedExchange (58) +ReusedExchange (55) -(58) ReusedExchange [Reuses operator id: 13] +(55) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 419706064ea82..9a090e0da62c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -30,7 +30,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ InputAdapter Exchange [c_customer_sk] #4 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -41,12 +41,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -63,21 +62,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (7) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (9) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 12f35bdc6bb56..ffb07d398de71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,58 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : :- * BroadcastHashJoin LeftSemi BuildRight (15) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * BroadcastHashJoin LeftSemi BuildRight (27) + : : :- * BroadcastHashJoin LeftSemi BuildRight (14) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.customer (1) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.store_sales (4) - : : : +- BroadcastExchange (11) - : : : +- * Project (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.date_dim (7) - : : +- BroadcastExchange (29) - : : +- Union (28) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Filter (18) - : : : : +- * ColumnarToRow (17) - : : : : +- Scan parquet default.web_sales (16) - : : : +- ReusedExchange (19) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer_address (32) - +- BroadcastExchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + : : : +- BroadcastExchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * ColumnarToRow (5) + : : : : +- Scan parquet default.store_sales (4) + : : : +- BroadcastExchange (10) + : : : +- * Project (9) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.date_dim (6) + : : +- BroadcastExchange (26) + : : +- Union (25) + : : :- * Project (19) + : : : +- * BroadcastHashJoin Inner BuildRight (18) + : : : :- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (17) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * ColumnarToRow (21) + : : : +- Scan parquet default.catalog_sales (20) + : : +- ReusedExchange (22) + : +- BroadcastExchange (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- BroadcastExchange (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.customer_demographics (35) (1) Scan parquet default.customer Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_customer_sk)] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 9] @@ -60,219 +57,204 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (3) Filter [codegen id : 9] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : ((isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_customer_sk#1)) +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) (4) Scan parquet default.store_sales Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 2] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(6) Filter [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Condition : isnotnull(ss_customer_sk#4) - -(7) Scan parquet default.date_dim +(6) Scan parquet default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(8) ColumnarToRow [codegen id : 1] +(7) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -(9) Filter [codegen id : 1] +(8) Filter [codegen id : 1] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(10) Project [codegen id : 1] +(9) Project [codegen id : 1] Output [1]: [d_date_sk#7] Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -(11) BroadcastExchange +(10) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] -(12) BroadcastHashJoin [codegen id : 2] +(11) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join condition: None -(13) Project [codegen id : 2] +(12) Project [codegen id : 2] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -(14) BroadcastExchange +(13) BroadcastExchange Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] -(15) BroadcastHashJoin [codegen id : 9] +(14) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join condition: None -(16) Scan parquet default.web_sales +(15) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] - -(18) Filter [codegen id : 4] +(16) ColumnarToRow [codegen id : 4] Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Condition : isnotnull(ws_bill_customer_sk#12) -(19) ReusedExchange [Reuses operator id: 11] +(17) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(20) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] Right keys [1]: [d_date_sk#7] Join condition: None -(21) Project [codegen id : 4] +(19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12 AS customsk#14] Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] -(22) Scan parquet default.catalog_sales +(20) Scan parquet default.catalog_sales Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_ship_customer_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] - -(24) Filter [codegen id : 6] +(21) ColumnarToRow [codegen id : 6] Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_ship_customer_sk#15) -(25) ReusedExchange [Reuses operator id: 11] +(22) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -(26) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#7] Join condition: None -(27) Project [codegen id : 6] +(24) Project [codegen id : 6] Output [1]: [cs_ship_customer_sk#15 AS customsk#17] Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] -(28) Union +(25) Union -(29) BroadcastExchange +(26) BroadcastExchange Input [1]: [customsk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] -(30) BroadcastHashJoin [codegen id : 9] +(27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#14] Join condition: None -(31) Project [codegen id : 9] +(28) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(32) Scan parquet default.customer_address +(29) 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_address_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 7] +(30) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#19, ca_state#20] -(34) Filter [codegen id : 7] +(31) Filter [codegen id : 7] Input [2]: [ca_address_sk#19, ca_state#20] Condition : isnotnull(ca_address_sk#19) -(35) BroadcastExchange +(32) BroadcastExchange Input [2]: [ca_address_sk#19, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(36) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#19] Join condition: None -(37) Project [codegen id : 9] +(34) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#2, ca_state#20] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] -(38) Scan parquet default.customer_demographics +(35) Scan parquet default.customer_demographics Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 8] +(36) ColumnarToRow [codegen id : 8] Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(40) Filter [codegen id : 8] +(37) Filter [codegen id : 8] Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Condition : isnotnull(cd_demo_sk#22) -(41) BroadcastExchange +(38) BroadcastExchange Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(42) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#22] Join condition: None -(43) Project [codegen id : 9] +(40) Project [codegen id : 9] Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(44) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 9] Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(45) Exchange +(42) Exchange Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] -(46) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] -(47) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -ReusedExchange (48) +ReusedExchange (45) -(48) ReusedExchange [Reuses operator id: 11] +(45) ReusedExchange [Reuses operator id: 10] Output [1]: [d_date_sk#7] -Subquery:2 Hosting operator id = 16 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index ae82b13a9533e..1ab7d548e59dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] @@ -21,12 +21,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (2) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,21 +40,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (4) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 849d2f418a9a8..5374a708a0295 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (95) -+- * HashAggregate (94) - +- Exchange (93) - +- * HashAggregate (92) - +- Union (91) - :- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- Union (77) +TakeOrderedAndProject (94) ++- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- Union (90) + :- * HashAggregate (79) + : +- Exchange (78) + : +- * HashAggregate (77) + : +- Union (76) : :- * HashAggregate (25) : : +- Exchange (24) : : +- * HashAggregate (23) @@ -54,46 +54,45 @@ TakeOrderedAndProject (95) : : : +- * ColumnarToRow (36) : : : +- Scan parquet default.catalog_page (35) : : +- ReusedExchange (41) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- Union (64) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * HashAggregate (73) + : +- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- Union (63) : : : :- * Project (50) : : : : +- * Filter (49) : : : : +- * ColumnarToRow (48) : : : : +- Scan parquet default.web_sales (47) - : : : +- * Project (63) - : : : +- * SortMergeJoin Inner (62) - : : : :- * Sort (55) - : : : : +- Exchange (54) - : : : : +- * Filter (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet default.web_returns (51) - : : : +- * Sort (61) - : : : +- Exchange (60) - : : : +- * Project (59) - : : : +- * Filter (58) - : : : +- * ColumnarToRow (57) - : : : +- Scan parquet default.web_sales (56) - : : +- BroadcastExchange (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.web_site (65) - : +- ReusedExchange (71) - :- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * HashAggregate (82) - : +- ReusedExchange (81) - +- * HashAggregate (90) - +- Exchange (89) - +- * HashAggregate (88) - +- * HashAggregate (87) - +- ReusedExchange (86) + : : : +- * Project (62) + : : : +- * SortMergeJoin Inner (61) + : : : :- * Sort (54) + : : : : +- Exchange (53) + : : : : +- * ColumnarToRow (52) + : : : : +- Scan parquet default.web_returns (51) + : : : +- * Sort (60) + : : : +- Exchange (59) + : : : +- * Project (58) + : : : +- * Filter (57) + : : : +- * ColumnarToRow (56) + : : : +- Scan parquet default.web_sales (55) + : : +- BroadcastExchange (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.web_site (64) + : +- ReusedExchange (70) + :- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * HashAggregate (81) + : +- ReusedExchange (80) + +- * HashAggregate (89) + +- Exchange (88) + +- * HashAggregate (87) + +- * HashAggregate (86) + +- ReusedExchange (85) (1) Scan parquet default.store_sales @@ -332,234 +331,229 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) - -(54) Exchange +(53) Exchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] -(55) Sort [codegen id : 15] +(54) Sort [codegen id : 15] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 -(56) Scan parquet default.web_sales +(55) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 16] +(56) ColumnarToRow [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(58) Filter [codegen id : 16] +(57) Filter [codegen id : 16] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(59) Project [codegen id : 16] +(58) Project [codegen id : 16] Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(60) Exchange +(59) Exchange Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] -(61) Sort [codegen id : 17] +(60) Sort [codegen id : 17] Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 -(62) SortMergeJoin [codegen id : 18] +(61) SortMergeJoin [codegen id : 18] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(63) Project [codegen id : 18] +(62) Project [codegen id : 18] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(64) Union +(63) Union -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [2]: [web_site_sk#113, web_site_id#114] Condition : isnotnull(web_site_sk#113) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#113, web_site_id#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] -(69) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#113] Join condition: None -(70) Project [codegen id : 21] +(69) Project [codegen id : 21] Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] -(71) ReusedExchange [Reuses operator id: 20] +(70) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] -(72) BroadcastHashJoin [codegen id : 21] +(71) BroadcastHashJoin [codegen id : 21] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None -(73) Project [codegen id : 21] +(72) Project [codegen id : 21] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] -(74) HashAggregate [codegen id : 21] +(73) HashAggregate [codegen id : 21] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] Keys [1]: [web_site_id#114] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -(75) Exchange +(74) Exchange Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] -(76) HashAggregate [codegen id : 22] +(75) HashAggregate [codegen id : 22] Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] Keys [1]: [web_site_id#114] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] -(77) Union +(76) Union -(78) HashAggregate [codegen id : 23] +(77) HashAggregate [codegen id : 23] Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -(79) Exchange +(78) Exchange Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] -(80) HashAggregate [codegen id : 24] +(79) HashAggregate [codegen id : 24] Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] -(81) ReusedExchange [Reuses operator id: unknown] +(80) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -(82) HashAggregate [codegen id : 48] +(81) HashAggregate [codegen id : 48] Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] -(83) HashAggregate [codegen id : 48] +(82) HashAggregate [codegen id : 48] Input [4]: [channel#41, sales#163, returns#164, profit#165] Keys [1]: [channel#41] Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] Aggregate Attributes [6]: [sum#166, isEmpty#167, sum#168, isEmpty#169, sum#170, isEmpty#171] Results [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -(84) Exchange +(83) Exchange Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] -(85) HashAggregate [codegen id : 49] +(84) HashAggregate [codegen id : 49] Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Keys [1]: [channel#41] Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] Aggregate Attributes [3]: [sum(sales#163)#179, sum(returns#164)#180, sum(profit#165)#181] Results [5]: [channel#41, null AS id#182, sum(sales#163)#179 AS sum(sales)#183, sum(returns#164)#180 AS sum(returns)#184, sum(profit#165)#181 AS sum(profit)#185] -(86) ReusedExchange [Reuses operator id: unknown] +(85) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -(87) HashAggregate [codegen id : 73] +(86) HashAggregate [codegen id : 73] Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] -(88) HashAggregate [codegen id : 73] +(87) HashAggregate [codegen id : 73] Input [3]: [sales#163, returns#164, profit#165] Keys: [] Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] -(89) Exchange +(88) Exchange Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] -(90) HashAggregate [codegen id : 74] +(89) HashAggregate [codegen id : 74] Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Keys: [] Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] -(91) Union +(90) Union -(92) HashAggregate [codegen id : 75] +(91) HashAggregate [codegen id : 75] Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -(93) Exchange +(92) Exchange Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] -(94) HashAggregate [codegen id : 76] +(93) HashAggregate [codegen id : 76] Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -(95) TakeOrderedAndProject +(94) TakeOrderedAndProject Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (96) +ReusedExchange (95) -(96) ReusedExchange [Reuses operator id: 20] +(95) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (97) +ReusedExchange (96) -(97) ReusedExchange [Reuses operator id: 20] +(96) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index 9a3d775ea8266..dd2c25b96568b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -121,11 +121,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Exchange [wr_item_sk,wr_order_number] #9 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index a5e7d0339212f..89362fb4f0efe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (92) -+- * HashAggregate (91) - +- Exchange (90) - +- * HashAggregate (89) - +- Union (88) - :- * HashAggregate (77) - : +- Exchange (76) - : +- * HashAggregate (75) - : +- Union (74) +TakeOrderedAndProject (91) ++- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- Union (87) + :- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- Union (73) : :- * HashAggregate (25) : : +- Exchange (24) : : +- * HashAggregate (23) @@ -54,43 +54,42 @@ TakeOrderedAndProject (92) : : +- * Filter (40) : : +- * ColumnarToRow (39) : : +- Scan parquet default.catalog_page (38) - : +- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- Union (61) + : +- * HashAggregate (72) + : +- Exchange (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- Union (60) : : : :- * Project (50) : : : : +- * Filter (49) : : : : +- * ColumnarToRow (48) : : : : +- Scan parquet default.web_sales (47) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildLeft (59) - : : : :- BroadcastExchange (54) - : : : : +- * Filter (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet default.web_returns (51) - : : : +- * Project (58) - : : : +- * Filter (57) - : : : +- * ColumnarToRow (56) - : : : +- Scan parquet default.web_sales (55) - : : +- ReusedExchange (62) - : +- BroadcastExchange (68) - : +- * Filter (67) - : +- * ColumnarToRow (66) - : +- Scan parquet default.web_site (65) - :- * HashAggregate (82) - : +- Exchange (81) - : +- * HashAggregate (80) - : +- * HashAggregate (79) - : +- ReusedExchange (78) - +- * HashAggregate (87) - +- Exchange (86) - +- * HashAggregate (85) - +- * HashAggregate (84) - +- ReusedExchange (83) + : : : +- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildLeft (58) + : : : :- BroadcastExchange (53) + : : : : +- * ColumnarToRow (52) + : : : : +- Scan parquet default.web_returns (51) + : : : +- * Project (57) + : : : +- * Filter (56) + : : : +- * ColumnarToRow (55) + : : : +- Scan parquet default.web_sales (54) + : : +- ReusedExchange (61) + : +- BroadcastExchange (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.web_site (64) + :- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * HashAggregate (78) + : +- ReusedExchange (77) + +- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- * HashAggregate (83) + +- ReusedExchange (82) (1) Scan parquet default.store_sales @@ -329,222 +328,217 @@ Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#1 Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -(53) Filter [codegen id : 14] +(53) BroadcastExchange Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Condition : (isnotnull(wr_item_sk#98) AND isnotnull(wr_order_number#99)) +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] -(54) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false], input[1, bigint, false]),false), [id=#103] - -(55) Scan parquet default.web_sales +(54) Scan parquet default.web_sales Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(56) ColumnarToRow +(55) ColumnarToRow Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(57) Filter +(56) Filter Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) -(58) Project +(57) Project Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -(59) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 15] Left keys [2]: [wr_item_sk#98, wr_order_number#99] Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None -(60) Project [codegen id : 15] +(59) Project [codegen id : 15] Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -(61) Union +(60) Union -(62) ReusedExchange [Reuses operator id: 14] +(61) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] -(63) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 18] Left keys [1]: [date_sk#93] Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None -(64) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] -(65) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 17] +(65) ColumnarToRow [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] -(67) Filter [codegen id : 17] +(66) Filter [codegen id : 17] Input [2]: [web_site_sk#112, web_site_id#113] Condition : isnotnull(web_site_sk#112) -(68) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#112, web_site_id#113] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] -(69) BroadcastHashJoin [codegen id : 18] +(68) BroadcastHashJoin [codegen id : 18] Left keys [1]: [wsr_web_site_sk#92] Right keys [1]: [web_site_sk#112] Join condition: None -(70) Project [codegen id : 18] +(69) Project [codegen id : 18] Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] -(71) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 18] Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] Keys [1]: [web_site_id#113] Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -(72) Exchange +(71) Exchange Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] -(73) HashAggregate [codegen id : 19] +(72) HashAggregate [codegen id : 19] Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] Keys [1]: [web_site_id#113] Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] -(74) Union +(73) Union -(75) HashAggregate [codegen id : 20] +(74) HashAggregate [codegen id : 20] Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -(76) Exchange +(75) Exchange Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] -(77) HashAggregate [codegen id : 21] +(76) HashAggregate [codegen id : 21] Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] -(78) ReusedExchange [Reuses operator id: unknown] +(77) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] -(79) HashAggregate [codegen id : 42] +(78) HashAggregate [codegen id : 42] Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] -(80) HashAggregate [codegen id : 42] +(79) HashAggregate [codegen id : 42] Input [4]: [channel#41, sales#162, returns#163, profit#164] Keys [1]: [channel#41] Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Results [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -(81) Exchange +(80) Exchange Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] -(82) HashAggregate [codegen id : 43] +(81) HashAggregate [codegen id : 43] Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys [1]: [channel#41] Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] Aggregate Attributes [3]: [sum(sales#162)#178, sum(returns#163)#179, sum(profit#164)#180] Results [5]: [channel#41, null AS id#181, sum(sales#162)#178 AS sum(sales)#182, sum(returns#163)#179 AS sum(returns)#183, sum(profit#164)#180 AS sum(profit)#184] -(83) ReusedExchange [Reuses operator id: unknown] +(82) ReusedExchange [Reuses operator id: unknown] Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -(84) HashAggregate [codegen id : 64] +(83) HashAggregate [codegen id : 64] Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] -(85) HashAggregate [codegen id : 64] +(84) HashAggregate [codegen id : 64] Input [3]: [sales#162, returns#163, profit#164] Keys: [] Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -(86) Exchange +(85) Exchange Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] -(87) HashAggregate [codegen id : 65] +(86) HashAggregate [codegen id : 65] Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] -(88) Union +(87) Union -(89) HashAggregate [codegen id : 66] +(88) HashAggregate [codegen id : 66] Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -(90) Exchange +(89) Exchange Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] -(91) HashAggregate [codegen id : 67] +(90) HashAggregate [codegen id : 67] Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -(92) TakeOrderedAndProject +(91) TakeOrderedAndProject Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cast(ss_sold_date_sk#4 as bigint) IN dynamicpruning#5 -ReusedExchange (93) +ReusedExchange (92) -(93) ReusedExchange [Reuses operator id: 14] +(92) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#49 IN dynamicpruning#50 -ReusedExchange (94) +ReusedExchange (93) -(94) ReusedExchange [Reuses operator id: 14] +(93) ReusedExchange [Reuses operator id: 14] Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index a787e89a028aa..caa7fcf4957ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -118,11 +118,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (14) - Filter [wr_item_sk,wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 09545881f17c3..3c65529504320 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -111,7 +111,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -119,7 +119,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index 00cad8c468868..c1b7cfef5638c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index de78b4ff4cdcc..406acb0e0a27f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -111,7 +111,7 @@ Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Output [3]: [s_store_sk#8, s_county#9, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_state)] +PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 8] @@ -119,7 +119,7 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] (13) Filter [codegen id : 8] Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_state#10)) +Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : ((isnotnull(ranking#19) AND (ranking#19 <= 5)) AND isnotnull(s_state#16)) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index b6fdfb5868b9c..cd1e7e69e65d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #5 WholeStageCodegen (8) BroadcastHashJoin [s_state,s_state] - Filter [s_store_sk,s_state] + Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_county,s_state] @@ -51,7 +51,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (7) Project [s_state] - Filter [ranking,s_state] + Filter [ranking] InputAdapter Window [_w2,s_state] WholeStageCodegen (6)