diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index fcf222c8fdab0..738535940c9a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1286,6 +1286,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val COLLAPSE_AGGREGATE_NODES_ENABLED = buildConf("spark.sql.execution.collapseAggregateNodes") + .internal() + .doc("Whether to collapse the Partial and the Final aggregate exec nodes based " + + "on whether there is exchange between them") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + val REMOVE_REDUNDANT_SORTS_ENABLED = buildConf("spark.sql.execution.removeRedundantSorts") .internal() .doc("Whether to remove redundant physical sort node") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CollapseAggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollapseAggregates.scala new file mode 100644 index 0000000000000..6eef207a54830 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollapseAggregates.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Final, Partial} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.internal.SQLConf + +/** + * Collapse Physical aggregate exec nodes together if there is no exchange between them and they + * correspond to Partial and Final Aggregation for same + * [[org.apache.spark.sql.catalyst.plans.logical.Aggregate]] logical node. + */ +object CollapseAggregates extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.COLLAPSE_AGGREGATE_NODES_ENABLED)) { + plan + } else { + collapseAggregates(plan) + } + } + + private def collapseAggregates(plan: SparkPlan): SparkPlan = { + plan transform { + case parent@HashAggregateExec(_, _, _, _, _, _, child: HashAggregateExec) + if checkIfAggregatesCanBeCollapsed(parent, child) => + val completeAggregateExpressions = child.aggregateExpressions.map(_.copy(mode = Complete)) + HashAggregateExec( + requiredChildDistributionExpressions = Some(child.groupingExpressions), + groupingExpressions = child.groupingExpressions, + aggregateExpressions = completeAggregateExpressions, + aggregateAttributes = completeAggregateExpressions.map(_.resultAttribute), + initialInputBufferOffset = 0, + resultExpressions = parent.resultExpressions, + child = child.child) + + case parent@SortAggregateExec(_, _, _, _, _, _, child: SortAggregateExec) + if checkIfAggregatesCanBeCollapsed(parent, child) => + val completeAggregateExpressions = child.aggregateExpressions.map(_.copy(mode = Complete)) + SortAggregateExec( + requiredChildDistributionExpressions = Some(child.groupingExpressions), + groupingExpressions = child.groupingExpressions, + aggregateExpressions = completeAggregateExpressions, + aggregateAttributes = completeAggregateExpressions.map(_.resultAttribute), + initialInputBufferOffset = 0, + resultExpressions = parent.resultExpressions, + child = child.child) + + case parent@ObjectHashAggregateExec(_, _, _, _, _, _, child: ObjectHashAggregateExec) + if checkIfAggregatesCanBeCollapsed(parent, child) => + val completeAggregateExpressions = child.aggregateExpressions.map(_.copy(mode = Complete)) + ObjectHashAggregateExec( + requiredChildDistributionExpressions = Some(child.groupingExpressions), + groupingExpressions = child.groupingExpressions, + aggregateExpressions = completeAggregateExpressions, + aggregateAttributes = completeAggregateExpressions.map(_.resultAttribute), + initialInputBufferOffset = 0, + resultExpressions = parent.resultExpressions, + child = child.child) + } + } + + private def checkIfAggregatesCanBeCollapsed( + parent: BaseAggregateExec, + child: BaseAggregateExec): Boolean = { + val parentHasFinalMode = parent.aggregateExpressions.forall(_.mode == Final) + if (!parentHasFinalMode) { + return false + } + val childHasPartialMode = child.aggregateExpressions.forall(_.mode == Partial) + if (!childHasPartialMode) { + return false + } + val parentChildAggExpressionsSame = parent.aggregateExpressions.map( + _.copy(mode = Partial)) == child.aggregateExpressions + if (!parentChildAggExpressionsSame) { + return false + } + true + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 040d1f36ed8a5..048363cb7b121 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -347,6 +347,7 @@ object QueryExecution { // `RemoveRedundantSorts` needs to be added before `EnsureRequirements` to guarantee the same // number of partitions when instantiating PartitioningCollection. RemoveRedundantSorts, + CollapseAggregates, DisableUnnecessaryBucketedScan, ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.columnarRules), CollapseCodegenStages(), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 570edbf5f78a3..31907b30a40f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -90,6 +90,7 @@ case class AdaptiveSparkPlanExec( RemoveRedundantProjects, EnsureRequirements, RemoveRedundantSorts, + CollapseAggregates, DisableUnnecessaryBucketedScan ) ++ context.session.sessionState.queryStagePrepRules diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index 7e82e34c4c637..f3475fa857ec4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -1,55 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * HashAggregate (35) - : : +- * HashAggregate (34) - : : +- * Project (33) - : : +- * SortMergeJoin Inner (32) - : : :- * Sort (26) - : : : +- Exchange (25) - : : : +- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet default.store_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Project (7) - : : : : : +- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.date_dim (4) - : : : : +- BroadcastExchange (15) - : : : : +- * Project (14) - : : : : +- * Filter (13) - : : : : +- * ColumnarToRow (12) - : : : : +- Scan parquet default.store (11) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.household_demographics (18) - : : +- * Sort (31) - : : +- Exchange (30) - : : +- * Filter (29) - : : +- * ColumnarToRow (28) - : : +- Scan parquet default.customer_address (27) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer (38) - +- * Sort (48) - +- ReusedExchange (47) +TakeOrderedAndProject (50) ++- * Project (49) + +- * SortMergeJoin Inner (48) + :- * Sort (45) + : +- Exchange (44) + : +- * Project (43) + : +- * SortMergeJoin Inner (42) + : :- * Sort (36) + : : +- Exchange (35) + : : +- * HashAggregate (34) + : : +- * Project (33) + : : +- * SortMergeJoin Inner (32) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Project (17) + : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.store_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.date_dim (4) + : : : : +- BroadcastExchange (15) + : : : : +- * Project (14) + : : : : +- * Filter (13) + : : : : +- * ColumnarToRow (12) + : : : : +- Scan parquet default.store (11) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet default.household_demographics (18) + : : +- * Sort (31) + : : +- Exchange (30) + : : +- * Filter (29) + : : +- * ColumnarToRow (28) + : : +- Scan parquet default.customer_address (27) + : +- * Sort (41) + : +- Exchange (40) + : +- * Filter (39) + : +- * ColumnarToRow (38) + : +- Scan parquet default.customer (37) + +- * Sort (47) + +- ReusedExchange (46) (1) Scan parquet default.store_sales @@ -201,81 +200,74 @@ Input [7]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, (34) HashAggregate [codegen id : 8] Input [6]: [ss_customer_sk#2, ss_addr_sk#4, ss_ticket_number#6, ss_coupon_amt#7, ss_net_profit#8, ca_city#22] Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#7)), partial_sum(UnscaledValue(ss_net_profit#8))] -Aggregate Attributes [2]: [sum#24, sum#25] -Results [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22, sum#26, sum#27] - -(35) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22, sum#26, sum#27] -Keys [4]: [ss_ticket_number#6, ss_customer_sk#2, ss_addr_sk#4, ca_city#22] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#7)), sum(UnscaledValue(ss_net_profit#8))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#7))#28, sum(UnscaledValue(ss_net_profit#8))#29] -Results [5]: [ss_ticket_number#6, ss_customer_sk#2, ca_city#22 AS bought_city#30, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#7))#28,17,2) AS amt#31, MakeDecimal(sum(UnscaledValue(ss_net_profit#8))#29,17,2) AS profit#32] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#7))#24, sum(UnscaledValue(ss_net_profit#8))#25] +Results [5]: [ss_ticket_number#6, ss_customer_sk#2, ca_city#22 AS bought_city#26, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#7))#24,17,2) AS amt#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#8))#25,17,2) AS profit#28] -(36) Exchange -Input [5]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#30, amt#31, profit#32] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#33] +(35) Exchange +Input [5]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#26, amt#27, profit#28] +Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#29] -(37) Sort [codegen id : 9] -Input [5]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#30, amt#31, profit#32] +(36) Sort [codegen id : 9] +Input [5]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#26, amt#27, profit#28] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer -Output [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] +(37) Scan parquet default.customer +Output [4]: [c_customer_sk#30, c_current_addr_sk#31, c_first_name#32, c_last_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] +(38) ColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#30, c_current_addr_sk#31, c_first_name#32, c_last_name#33] -(40) Filter [codegen id : 10] -Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#35)) +(39) Filter [codegen id : 10] +Input [4]: [c_customer_sk#30, c_current_addr_sk#31, c_first_name#32, c_last_name#33] +Condition : (isnotnull(c_customer_sk#30) AND isnotnull(c_current_addr_sk#31)) -(41) Exchange -Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -Arguments: hashpartitioning(c_customer_sk#34, 5), true, [id=#38] +(40) Exchange +Input [4]: [c_customer_sk#30, c_current_addr_sk#31, c_first_name#32, c_last_name#33] +Arguments: hashpartitioning(c_customer_sk#30, 5), true, [id=#34] -(42) Sort [codegen id : 11] -Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 11] +Input [4]: [c_customer_sk#30, c_current_addr_sk#31, c_first_name#32, c_last_name#33] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 12] +(42) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#34] +Right keys [1]: [c_customer_sk#30] Join condition: None -(44) Project [codegen id : 12] -Output [7]: [ss_ticket_number#6, bought_city#30, amt#31, profit#32, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -Input [9]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#30, amt#31, profit#32, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] +(43) Project [codegen id : 12] +Output [7]: [ss_ticket_number#6, bought_city#26, amt#27, profit#28, c_current_addr_sk#31, c_first_name#32, c_last_name#33] +Input [9]: [ss_ticket_number#6, ss_customer_sk#2, bought_city#26, amt#27, profit#28, c_customer_sk#30, c_current_addr_sk#31, c_first_name#32, c_last_name#33] -(45) Exchange -Input [7]: [ss_ticket_number#6, bought_city#30, amt#31, profit#32, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -Arguments: hashpartitioning(c_current_addr_sk#35, 5), true, [id=#39] +(44) Exchange +Input [7]: [ss_ticket_number#6, bought_city#26, amt#27, profit#28, c_current_addr_sk#31, c_first_name#32, c_last_name#33] +Arguments: hashpartitioning(c_current_addr_sk#31, 5), true, [id=#35] -(46) Sort [codegen id : 13] -Input [7]: [ss_ticket_number#6, bought_city#30, amt#31, profit#32, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -Arguments: [c_current_addr_sk#35 ASC NULLS FIRST], false, 0 +(45) Sort [codegen id : 13] +Input [7]: [ss_ticket_number#6, bought_city#26, amt#27, profit#28, c_current_addr_sk#31, c_first_name#32, c_last_name#33] +Arguments: [c_current_addr_sk#31 ASC NULLS FIRST], false, 0 -(47) ReusedExchange [Reuses operator id: 30] +(46) ReusedExchange [Reuses operator id: 30] Output [2]: [ca_address_sk#21, ca_city#22] -(48) Sort [codegen id : 15] +(47) Sort [codegen id : 15] Input [2]: [ca_address_sk#21, ca_city#22] Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#35] +(48) SortMergeJoin [codegen id : 16] +Left keys [1]: [c_current_addr_sk#31] Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#30) +Join condition: NOT (ca_city#22 = bought_city#26) -(50) Project [codegen id : 16] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#30, ss_ticket_number#6, amt#31, profit#32] -Input [9]: [ss_ticket_number#6, bought_city#30, amt#31, profit#32, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +(49) Project [codegen id : 16] +Output [7]: [c_last_name#33, c_first_name#32, ca_city#22, bought_city#26, ss_ticket_number#6, amt#27, profit#28] +Input [9]: [ss_ticket_number#6, bought_city#26, amt#27, profit#28, c_current_addr_sk#31, c_first_name#32, c_last_name#33, ca_address_sk#21, ca_city#22] -(51) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#30, ss_ticket_number#6, amt#31, profit#32] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#30 ASC NULLS FIRST, ss_ticket_number#6 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#30, ss_ticket_number#6, amt#31, profit#32] +(50) TakeOrderedAndProject +Input [7]: [c_last_name#33, c_first_name#32, ca_city#22, bought_city#26, ss_ticket_number#6, amt#27, profit#28] +Arguments: 100, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#26 ASC NULLS FIRST, ss_ticket_number#6 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, ca_city#22, bought_city#26, ss_ticket_number#6, amt#27, profit#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt index bc83e3b7e2eba..47730c74f7f9c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt @@ -16,60 +16,59 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu InputAdapter Exchange [ss_customer_sk] #2 WholeStageCodegen (8) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - SortMergeJoin [ss_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (5) - Sort [ss_addr_sk] - InputAdapter - Exchange [ss_addr_sk] #3 - WholeStageCodegen (4) - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_dow,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [s_store_sk] - Filter [s_city,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - WholeStageCodegen (7) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #7 - WholeStageCodegen (6) - Filter [ca_address_sk,ca_city] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + SortMergeJoin [ss_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_addr_sk] + InputAdapter + Exchange [ss_addr_sk] #3 + WholeStageCodegen (4) + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_dow,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Project [s_store_sk] + Filter [s_city,s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_city] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [hd_demo_sk] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + WholeStageCodegen (7) + Sort [ca_address_sk] + InputAdapter + Exchange [ca_address_sk] #7 + WholeStageCodegen (6) + Filter [ca_address_sk,ca_city] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_city] InputAdapter WholeStageCodegen (11) Sort [c_customer_sk] 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 b346701fa3148..912a0ccfd123a 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 @@ -1,135 +1,134 @@ == Physical Plan == -TakeOrderedAndProject (131) -+- * HashAggregate (130) - +- Exchange (129) - +- * HashAggregate (128) - +- * Expand (127) - +- Union (126) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (130) ++- * HashAggregate (129) + +- Exchange (128) + +- * HashAggregate (127) + +- * Expand (126) + +- Union (125) + :- * Project (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * Project (81) + : +- * BroadcastHashJoin Inner BuildRight (80) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- SortMergeJoin LeftSemi (63) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (62) + : : : +- Exchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - :- * Project (106) - : +- * Filter (105) - : +- * HashAggregate (104) - : +- Exchange (103) - : +- * HashAggregate (102) - : +- * Project (101) - : +- * BroadcastHashJoin Inner BuildRight (100) - : :- * Project (98) - : : +- * BroadcastHashJoin Inner BuildRight (97) - : : :- SortMergeJoin LeftSemi (95) - : : : :- * Sort (92) - : : : : +- Exchange (91) - : : : : +- * Filter (90) - : : : : +- * ColumnarToRow (89) - : : : : +- Scan parquet default.catalog_sales (88) - : : : +- * Sort (94) - : : : +- ReusedExchange (93) - : : +- ReusedExchange (96) - : +- ReusedExchange (99) - +- * Project (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * Project (117) - : +- * BroadcastHashJoin Inner BuildRight (116) - : :- SortMergeJoin LeftSemi (114) - : : :- * Sort (111) - : : : +- Exchange (110) - : : : +- * Filter (109) - : : : +- * ColumnarToRow (108) - : : : +- Scan parquet default.web_sales (107) - : : +- * Sort (113) - : : +- ReusedExchange (112) - : +- ReusedExchange (115) - +- ReusedExchange (118) + : : : +- BroadcastExchange (58) + : : : +- * HashAggregate (57) + : : : +- * HashAggregate (56) + : : : +- Exchange (55) + : : : +- * HashAggregate (54) + : : : +- SortMergeJoin LeftSemi (53) + : : : :- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- BroadcastExchange (68) + : : +- * Project (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.date_dim (64) + : +- BroadcastExchange (79) + : +- SortMergeJoin LeftSemi (78) + : :- * Sort (75) + : : +- Exchange (74) + : : +- * Filter (73) + : : +- * ColumnarToRow (72) + : : +- Scan parquet default.item (71) + : +- * Sort (77) + : +- ReusedExchange (76) + :- * Project (105) + : +- * Filter (104) + : +- * HashAggregate (103) + : +- Exchange (102) + : +- * HashAggregate (101) + : +- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- SortMergeJoin LeftSemi (94) + : : : :- * Sort (91) + : : : : +- Exchange (90) + : : : : +- * Filter (89) + : : : : +- * ColumnarToRow (88) + : : : : +- Scan parquet default.catalog_sales (87) + : : : +- * Sort (93) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- ReusedExchange (98) + +- * Project (124) + +- * Filter (123) + +- * HashAggregate (122) + +- Exchange (121) + +- * HashAggregate (120) + +- * Project (119) + +- * BroadcastHashJoin Inner BuildRight (118) + :- * Project (116) + : +- * BroadcastHashJoin Inner BuildRight (115) + : :- SortMergeJoin LeftSemi (113) + : : :- * Sort (110) + : : : +- Exchange (109) + : : : +- * Filter (108) + : : : +- * ColumnarToRow (107) + : : : +- Scan parquet default.web_sales (106) + : : +- * Sort (112) + : : +- ReusedExchange (111) + : +- ReusedExchange (114) + +- ReusedExchange (117) (1) Scan parquet default.store_sales @@ -390,489 +389,482 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(58) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(60) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(61) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(62) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(63) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2001)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(67) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(70) BroadcastHashJoin [codegen id : 38] +(69) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(70) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(71) Scan parquet default.item 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_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(72) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(73) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(74) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#31] -(76) Sort [codegen id : 21] +(75) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(76) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(77) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(78) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(79) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(81) BroadcastHashJoin [codegen id : 38] +(80) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(81) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(82) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -(84) Exchange +(83) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#39] -(85) HashAggregate [codegen id : 39] +(84) HashAggregate [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(86) Filter [codegen id : 39] +(85) Filter [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(87) Project [codegen id : 39] +(86) Project [codegen id : 39] Output [6]: [sales#42, number_sales#43, store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(88) Scan parquet default.catalog_sales +(87) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] +(88) ColumnarToRow [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -(90) Filter [codegen id : 40] +(89) Filter [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange +(90) Exchange Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: hashpartitioning(cs_item_sk#19, 5), true, [id=#50] -(92) Sort [codegen id : 41] +(91) Sort [codegen id : 41] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] +(92) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(94) Sort [codegen id : 57] +(93) Sort [codegen id : 57] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(94) SortMergeJoin Left keys [1]: [cs_item_sk#19] Right keys [1]: [ss_item_sk#27] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(95) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] +(97) Project [codegen id : 77] Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(98) ReusedExchange [Reuses operator id: 79] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(99) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] +(100) Project [codegen id : 77] Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] +(101) HashAggregate [codegen id : 77] Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -(103) Exchange +(102) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#57] -(104) HashAggregate [codegen id : 78] +(103) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(105) Filter [codegen id : 78] +(104) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(106) Project [codegen id : 78] +(105) Project [codegen id : 78] Output [6]: [sales#60, number_sales#61, catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(107) Scan parquet default.web_sales +(106) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] +(107) ColumnarToRow [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -(109) Filter [codegen id : 79] +(108) Filter [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) -(110) Exchange +(109) Exchange Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: hashpartitioning(ws_item_sk#23, 5), true, [id=#66] -(111) Sort [codegen id : 80] +(110) Sort [codegen id : 80] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] +(111) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(113) Sort [codegen id : 96] +(112) Sort [codegen id : 96] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin +(113) SortMergeJoin Left keys [1]: [ws_item_sk#23] Right keys [1]: [ss_item_sk#27] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(114) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] +(115) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] +(116) Project [codegen id : 116] Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(117) ReusedExchange [Reuses operator id: 79] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] +(118) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_item_sk#23] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] +(119) Project [codegen id : 116] Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] +(120) HashAggregate [codegen id : 116] Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -(122) Exchange +(121) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#73] -(123) HashAggregate [codegen id : 117] +(122) HashAggregate [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(124) Filter [codegen id : 117] +(123) Filter [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(125) Project [codegen id : 117] +(124) Project [codegen id : 117] Output [6]: [sales#76, number_sales#77, web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(126) Union +(125) Union -(127) Expand [codegen id : 118] +(126) Expand [codegen id : 118] Input [6]: [sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, null, 1), List(sales#42, number_sales#43, channel#47, i_brand_id#7, null, null, 3), List(sales#42, number_sales#43, channel#47, null, null, null, 7), List(sales#42, number_sales#43, null, null, null, null, 15)], [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -(128) HashAggregate [codegen id : 118] +(127) HashAggregate [codegen id : 118] Input [7]: [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#85, isEmpty#86, sum#87] Results [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] -(129) Exchange +(128) Exchange Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] Arguments: hashpartitioning(channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, 5), true, [id=#91] -(130) HashAggregate [codegen id : 119] +(129) HashAggregate [codegen id : 119] Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#92, sum(number_sales#43)#93] Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#42)#92 AS sum(sales)#94, sum(number_sales#43)#93 AS sum(number_sales)#95] -(131) TakeOrderedAndProject +(130) TakeOrderedAndProject Input [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] Arguments: 100, [channel#80 ASC NULLS FIRST, i_brand_id#81 ASC NULLS FIRST, i_class_id#82 ASC NULLS FIRST, i_category_id#83 ASC NULLS FIRST], [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (157) -+- Exchange (156) - +- * HashAggregate (155) - +- Union (154) - :- * Project (141) - : +- * BroadcastHashJoin Inner BuildRight (140) - : :- * Filter (134) - : : +- * ColumnarToRow (133) - : : +- Scan parquet default.store_sales (132) - : +- BroadcastExchange (139) - : +- * Project (138) - : +- * Filter (137) - : +- * ColumnarToRow (136) - : +- Scan parquet default.date_dim (135) - :- * Project (147) - : +- * BroadcastHashJoin Inner BuildRight (146) - : :- * Filter (144) - : : +- * ColumnarToRow (143) - : : +- Scan parquet default.catalog_sales (142) - : +- ReusedExchange (145) - +- * Project (153) - +- * BroadcastHashJoin Inner BuildRight (152) - :- * Filter (150) - : +- * ColumnarToRow (149) - : +- Scan parquet default.web_sales (148) - +- ReusedExchange (151) - - -(132) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 85 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* HashAggregate (156) ++- Exchange (155) + +- * HashAggregate (154) + +- Union (153) + :- * Project (140) + : +- * BroadcastHashJoin Inner BuildRight (139) + : :- * Filter (133) + : : +- * ColumnarToRow (132) + : : +- Scan parquet default.store_sales (131) + : +- BroadcastExchange (138) + : +- * Project (137) + : +- * Filter (136) + : +- * ColumnarToRow (135) + : +- Scan parquet default.date_dim (134) + :- * Project (146) + : +- * BroadcastHashJoin Inner BuildRight (145) + : :- * Filter (143) + : : +- * ColumnarToRow (142) + : : +- Scan parquet default.catalog_sales (141) + : +- ReusedExchange (144) + +- * Project (152) + +- * BroadcastHashJoin Inner BuildRight (151) + :- * Filter (149) + : +- * ColumnarToRow (148) + : +- Scan parquet default.web_sales (147) + +- ReusedExchange (150) + + +(131) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] +(132) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(134) Filter [codegen id : 2] +(133) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(135) Scan parquet default.date_dim +(134) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(136) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(137) Filter [codegen id : 1] +(136) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(138) Project [codegen id : 1] +(137) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(139) BroadcastExchange +(138) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#96] -(140) BroadcastHashJoin [codegen id : 2] +(139) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(141) Project [codegen id : 2] +(140) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#97, ss_list_price#4 AS list_price#98] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(142) Scan parquet default.catalog_sales +(141) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(143) ColumnarToRow [codegen id : 4] +(142) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] -(144) Filter [codegen id : 4] +(143) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Condition : isnotnull(cs_sold_date_sk#18) -(145) ReusedExchange [Reuses operator id: 139] +(144) ReusedExchange [Reuses operator id: 138] Output [1]: [d_date_sk#10] -(146) BroadcastHashJoin [codegen id : 4] +(145) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(147) Project [codegen id : 4] +(146) Project [codegen id : 4] Output [2]: [cs_quantity#48 AS quantity#99, cs_list_price#49 AS list_price#100] Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(148) Scan parquet default.web_sales +(147) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(149) ColumnarToRow [codegen id : 6] +(148) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -(150) Filter [codegen id : 6] +(149) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Condition : isnotnull(ws_sold_date_sk#22) -(151) ReusedExchange [Reuses operator id: 139] +(150) ReusedExchange [Reuses operator id: 138] Output [1]: [d_date_sk#10] -(152) BroadcastHashJoin [codegen id : 6] +(151) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(153) Project [codegen id : 6] +(152) Project [codegen id : 6] Output [2]: [ws_quantity#64 AS quantity#101, ws_list_price#65 AS list_price#102] Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(154) Union +(153) Union -(155) HashAggregate [codegen id : 7] +(154) HashAggregate [codegen id : 7] Input [2]: [quantity#97, list_price#98] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#103, count#104] Results [2]: [sum#105, count#106] -(156) Exchange +(155) Exchange Input [2]: [sum#105, count#106] Arguments: SinglePartition, true, [id=#107] -(157) HashAggregate [codegen id : 8] +(156) HashAggregate [codegen id : 8] Input [2]: [sum#105, count#106] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108 AS average_sales#109] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] 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 5b93392d023db..e95aeaf3e6c2b 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 @@ -89,81 +89,80 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - Filter [i_item_sk,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] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (13) + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (9) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + BroadcastExchange #11 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter BroadcastExchange #13 WholeStageCodegen (19) 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 3f0cc9e7acb1e..8d949bc2753b2 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 @@ -1,119 +1,118 @@ == Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- Exchange (113) - +- * HashAggregate (112) - +- * Expand (111) - +- Union (110) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (114) ++- * HashAggregate (113) + +- Exchange (112) + +- * HashAggregate (111) + +- * Expand (110) + +- Union (109) + :- * Project (76) + : +- * Filter (75) + : +- * HashAggregate (74) + : +- Exchange (73) + : +- * HashAggregate (72) + : +- * Project (71) + : +- * BroadcastHashJoin Inner BuildRight (70) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * BroadcastHashJoin LeftSemi BuildRight (56) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (55) + : : : +- * Project (54) + : : : +- * BroadcastHashJoin Inner BuildRight (53) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - :- * Project (93) - : +- * Filter (92) - : +- * HashAggregate (91) - : +- Exchange (90) - : +- * HashAggregate (89) - : +- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Project (85) - : : +- * BroadcastHashJoin Inner BuildRight (84) - : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : :- * Filter (80) - : : : : +- * ColumnarToRow (79) - : : : : +- Scan parquet default.catalog_sales (78) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - +- * Project (109) - +- * Filter (108) - +- * HashAggregate (107) - +- Exchange (106) - +- * HashAggregate (105) - +- * Project (104) - +- * BroadcastHashJoin Inner BuildRight (103) - :- * Project (101) - : +- * BroadcastHashJoin Inner BuildRight (100) - : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : :- * Filter (96) - : : : +- * ColumnarToRow (95) - : : : +- Scan parquet default.web_sales (94) - : : +- ReusedExchange (97) - : +- ReusedExchange (99) - +- ReusedExchange (102) + : : : +- BroadcastExchange (52) + : : : +- * HashAggregate (51) + : : : +- * HashAggregate (50) + : : : +- Exchange (49) + : : : +- * HashAggregate (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (39) + : : : : : +- * ColumnarToRow (38) + : : : : : +- Scan parquet default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (62) + : : +- * BroadcastHashJoin LeftSemi BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.item (57) + : : +- ReusedExchange (60) + : +- BroadcastExchange (69) + : +- * Project (68) + : +- * Filter (67) + : +- * ColumnarToRow (66) + : +- Scan parquet default.date_dim (65) + :- * Project (92) + : +- * Filter (91) + : +- * HashAggregate (90) + : +- Exchange (89) + : +- * HashAggregate (88) + : +- * Project (87) + : +- * BroadcastHashJoin Inner BuildRight (86) + : :- * Project (84) + : : +- * BroadcastHashJoin Inner BuildRight (83) + : : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : : :- * Filter (79) + : : : : +- * ColumnarToRow (78) + : : : : +- Scan parquet default.catalog_sales (77) + : : : +- ReusedExchange (80) + : : +- ReusedExchange (82) + : +- ReusedExchange (85) + +- * Project (108) + +- * Filter (107) + +- * HashAggregate (106) + +- Exchange (105) + +- * HashAggregate (104) + +- * Project (103) + +- * BroadcastHashJoin Inner BuildRight (102) + :- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * BroadcastHashJoin LeftSemi BuildRight (97) + : : :- * Filter (95) + : : : +- * ColumnarToRow (94) + : : : +- Scan parquet default.web_sales (93) + : : +- ReusedExchange (96) + : +- ReusedExchange (98) + +- ReusedExchange (101) (1) Scan parquet default.store_sales @@ -350,449 +349,442 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(52) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(53) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(54) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(55) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(57) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(58) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(59) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : isnotnull(i_item_sk#5) -(61) ReusedExchange [Reuses operator id: 56] +(60) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(61) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(62) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(65) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(66) ColumnarToRow [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(68) Filter [codegen id : 24] +(67) Filter [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#28)) AND (d_year#11 = 2001)) AND (d_moy#28 = 11)) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(68) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(70) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] -(71) BroadcastHashJoin [codegen id : 25] +(70) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(71) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(72) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#30, isEmpty#31, count#32] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -(74) Exchange +(73) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#36] -(75) HashAggregate [codegen id : 26] +(74) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(76) Filter [codegen id : 26] +(75) Filter [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(77) Project [codegen id : 26] +(76) Project [codegen id : 26] Output [6]: [sales#39, number_sales#40, store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(78) Scan parquet default.catalog_sales +(77) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 51] +(78) ColumnarToRow [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -(80) Filter [codegen id : 51] +(79) Filter [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) -(81) ReusedExchange [Reuses operator id: 56] +(80) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 51] +(81) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(82) ReusedExchange [Reuses operator id: 62] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(84) BroadcastHashJoin [codegen id : 51] +(83) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [i_item_sk#5] Join condition: None -(85) Project [codegen id : 51] +(84) Project [codegen id : 51] Output [6]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(86) ReusedExchange [Reuses operator id: 70] +(85) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#10] -(87) BroadcastHashJoin [codegen id : 51] +(86) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(88) Project [codegen id : 51] +(87) Project [codegen id : 51] Output [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(89) HashAggregate [codegen id : 51] +(88) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -(90) Exchange +(89) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#53] -(91) HashAggregate [codegen id : 52] +(90) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(92) Filter [codegen id : 52] +(91) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(93) Project [codegen id : 52] +(92) Project [codegen id : 52] Output [6]: [sales#56, number_sales#57, catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(94) Scan parquet default.web_sales +(93) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 77] +(94) ColumnarToRow [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -(96) Filter [codegen id : 77] +(95) Filter [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) -(97) ReusedExchange [Reuses operator id: 56] +(96) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(98) BroadcastHashJoin [codegen id : 77] +(97) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [ss_item_sk#25] Join condition: None -(99) ReusedExchange [Reuses operator id: 63] +(98) ReusedExchange [Reuses operator id: 62] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(100) BroadcastHashJoin [codegen id : 77] +(99) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [i_item_sk#5] Join condition: None -(101) Project [codegen id : 77] +(100) Project [codegen id : 77] Output [6]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(102) ReusedExchange [Reuses operator id: 70] +(101) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#10] -(103) BroadcastHashJoin [codegen id : 77] +(102) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(104) Project [codegen id : 77] +(103) Project [codegen id : 77] Output [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(105) HashAggregate [codegen id : 77] +(104) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#62, isEmpty#63, count#64] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -(106) Exchange +(105) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#68] -(107) HashAggregate [codegen id : 78] +(106) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(108) Filter [codegen id : 78] +(107) Filter [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(109) Project [codegen id : 78] +(108) Project [codegen id : 78] Output [6]: [sales#71, number_sales#72, web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(110) Union +(109) Union -(111) Expand [codegen id : 79] +(110) Expand [codegen id : 79] Input [6]: [sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: [List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, null, 1), List(sales#39, number_sales#40, channel#44, i_brand_id#6, null, null, 3), List(sales#39, number_sales#40, channel#44, null, null, null, 7), List(sales#39, number_sales#40, null, null, null, null, 15)], [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -(112) HashAggregate [codegen id : 79] +(111) HashAggregate [codegen id : 79] Input [7]: [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] Results [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] -(113) Exchange +(112) Exchange Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] Arguments: hashpartitioning(channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, 5), true, [id=#86] -(114) HashAggregate [codegen id : 80] +(113) HashAggregate [codegen id : 80] Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#87, sum(number_sales#40)#88] Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#39)#87 AS sum(sales)#89, sum(number_sales#40)#88 AS sum(number_sales)#90] -(115) TakeOrderedAndProject +(114) TakeOrderedAndProject Input [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] Arguments: 100, [channel#75 ASC NULLS FIRST, i_brand_id#76 ASC NULLS FIRST, i_class_id#77 ASC NULLS FIRST, i_category_id#78 ASC NULLS FIRST], [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (141) -+- Exchange (140) - +- * HashAggregate (139) - +- Union (138) - :- * Project (125) - : +- * BroadcastHashJoin Inner BuildRight (124) - : :- * Filter (118) - : : +- * ColumnarToRow (117) - : : +- Scan parquet default.store_sales (116) - : +- BroadcastExchange (123) - : +- * Project (122) - : +- * Filter (121) - : +- * ColumnarToRow (120) - : +- Scan parquet default.date_dim (119) - :- * Project (131) - : +- * BroadcastHashJoin Inner BuildRight (130) - : :- * Filter (128) - : : +- * ColumnarToRow (127) - : : +- Scan parquet default.catalog_sales (126) - : +- ReusedExchange (129) - +- * Project (137) - +- * BroadcastHashJoin Inner BuildRight (136) - :- * Filter (134) - : +- * ColumnarToRow (133) - : +- Scan parquet default.web_sales (132) - +- ReusedExchange (135) - - -(116) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 75 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (140) ++- Exchange (139) + +- * HashAggregate (138) + +- Union (137) + :- * Project (124) + : +- * BroadcastHashJoin Inner BuildRight (123) + : :- * Filter (117) + : : +- * ColumnarToRow (116) + : : +- Scan parquet default.store_sales (115) + : +- BroadcastExchange (122) + : +- * Project (121) + : +- * Filter (120) + : +- * ColumnarToRow (119) + : +- Scan parquet default.date_dim (118) + :- * Project (130) + : +- * BroadcastHashJoin Inner BuildRight (129) + : :- * Filter (127) + : : +- * ColumnarToRow (126) + : : +- Scan parquet default.catalog_sales (125) + : +- ReusedExchange (128) + +- * Project (136) + +- * BroadcastHashJoin Inner BuildRight (135) + :- * Filter (133) + : +- * ColumnarToRow (132) + : +- Scan parquet default.web_sales (131) + +- ReusedExchange (134) + + +(115) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 2] +(116) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(118) Filter [codegen id : 2] +(117) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(119) Scan parquet default.date_dim +(118) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] +(119) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(121) Filter [codegen id : 1] +(120) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(122) Project [codegen id : 1] +(121) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(123) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#91] -(124) BroadcastHashJoin [codegen id : 2] +(123) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(125) Project [codegen id : 2] +(124) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#92, ss_list_price#4 AS list_price#93] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(126) Scan parquet default.catalog_sales +(125) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 4] +(126) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -(128) Filter [codegen id : 4] +(127) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Condition : isnotnull(cs_sold_date_sk#16) -(129) ReusedExchange [Reuses operator id: 123] +(128) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#10] -(130) BroadcastHashJoin [codegen id : 4] +(129) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(131) Project [codegen id : 4] +(130) Project [codegen id : 4] Output [2]: [cs_quantity#45 AS quantity#94, cs_list_price#46 AS list_price#95] Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] -(132) Scan parquet default.web_sales +(131) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 6] +(132) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -(134) Filter [codegen id : 6] +(133) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Condition : isnotnull(ws_sold_date_sk#20) -(135) ReusedExchange [Reuses operator id: 123] +(134) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#10] -(136) BroadcastHashJoin [codegen id : 6] +(135) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(137) Project [codegen id : 6] +(136) Project [codegen id : 6] Output [2]: [ws_quantity#60 AS quantity#96, ws_list_price#61 AS list_price#97] Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] -(138) Union +(137) Union -(139) HashAggregate [codegen id : 7] +(138) HashAggregate [codegen id : 7] Input [2]: [quantity#92, list_price#93] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#98, count#99] Results [2]: [sum#100, count#101] -(140) Exchange +(139) Exchange Input [2]: [sum#100, count#101] Arguments: SinglePartition, true, [id=#102] -(141) HashAggregate [codegen id : 8] +(140) HashAggregate [codegen id : 8] Input [2]: [sum#100, count#101] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103 AS average_sales#104] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 91 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:3 Hosting operator id = 107 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] 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 dfa8c1bcc1579..d2a8f2482b3fe 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 @@ -81,71 +81,70 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Filter [i_item_sk,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] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (5) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + BroadcastExchange #9 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (23) 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 2d2b56e32bdb8..6f1f6d0bb2458 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 @@ -1,114 +1,113 @@ == Physical Plan == -TakeOrderedAndProject (110) -+- * BroadcastHashJoin Inner BuildRight (109) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (109) ++- * BroadcastHashJoin Inner BuildRight (108) + :- * Project (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * Project (81) + : +- * BroadcastHashJoin Inner BuildRight (80) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- SortMergeJoin LeftSemi (63) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (62) + : : : +- Exchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - +- BroadcastExchange (108) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- SortMergeJoin LeftSemi (92) - : : :- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (91) - : : +- ReusedExchange (90) - : +- BroadcastExchange (97) - : +- * Project (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet default.date_dim (93) - +- ReusedExchange (100) + : : : +- BroadcastExchange (58) + : : : +- * HashAggregate (57) + : : : +- * HashAggregate (56) + : : : +- Exchange (55) + : : : +- * HashAggregate (54) + : : : +- SortMergeJoin LeftSemi (53) + : : : :- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- BroadcastExchange (68) + : : +- * Project (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.date_dim (64) + : +- BroadcastExchange (79) + : +- SortMergeJoin LeftSemi (78) + : :- * Sort (75) + : : +- Exchange (74) + : : +- * Filter (73) + : : +- * ColumnarToRow (72) + : : +- Scan parquet default.item (71) + : +- * Sort (77) + : +- ReusedExchange (76) + +- BroadcastExchange (107) + +- * Project (106) + +- * Filter (105) + +- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Project (101) + +- * BroadcastHashJoin Inner BuildRight (100) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- SortMergeJoin LeftSemi (91) + : : :- * Sort (88) + : : : +- ReusedExchange (87) + : : +- * Sort (90) + : : +- ReusedExchange (89) + : +- BroadcastExchange (96) + : +- * Project (95) + : +- * Filter (94) + : +- * ColumnarToRow (93) + : +- Scan parquet default.date_dim (92) + +- ReusedExchange (99) (1) Scan parquet default.store_sales @@ -369,441 +368,434 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(58) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(60) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(61) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(62) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(63) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#30, [id=#31])) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(67) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] -(70) BroadcastHashJoin [codegen id : 38] +(69) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(70) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(71) Scan parquet default.item 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_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(72) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(73) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(75) Exchange +(74) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#33] -(76) Sort [codegen id : 21] +(75) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(76) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(77) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(78) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(79) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] -(81) BroadcastHashJoin [codegen id : 38] +(80) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(81) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(82) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -(84) Exchange +(83) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#41] -(85) HashAggregate [codegen id : 78] +(84) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(86) Filter [codegen id : 78] +(85) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(87) Project [codegen id : 78] +(86) Project [codegen id : 78] Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(88) ReusedExchange [Reuses operator id: 4] +(87) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(89) Sort [codegen id : 40] +(88) Sort [codegen id : 40] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 62] +(89) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(91) Sort [codegen id : 56] +(90) Sort [codegen id : 56] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(91) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(93) Scan parquet default.date_dim +(92) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 57] +(93) ColumnarToRow [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] -(95) Filter [codegen id : 57] +(94) Filter [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 57] +(95) Project [codegen id : 57] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(97) BroadcastExchange +(96) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(98) BroadcastHashJoin [codegen id : 76] +(97) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(99) Project [codegen id : 76] +(98) Project [codegen id : 76] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(100) ReusedExchange [Reuses operator id: 80] +(99) ReusedExchange [Reuses operator id: 79] Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(101) BroadcastHashJoin [codegen id : 76] +(100) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#53] Join condition: None -(102) Project [codegen id : 76] +(101) Project [codegen id : 76] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(103) HashAggregate [codegen id : 76] +(102) HashAggregate [codegen id : 76] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -(104) Exchange +(103) Exchange Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), true, [id=#63] -(105) HashAggregate [codegen id : 77] +(104) HashAggregate [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(106) Filter [codegen id : 77] +(105) Filter [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(107) Project [codegen id : 77] +(106) Project [codegen id : 77] Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(108) BroadcastExchange +(107) BroadcastExchange Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] -(109) BroadcastHashJoin [codegen id : 78] +(108) BroadcastHashJoin [codegen id : 78] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Join condition: None -(110) TakeOrderedAndProject +(109) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (136) -+- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.store_sales (111) - : +- BroadcastExchange (118) - : +- * Project (117) - : +- * Filter (116) - : +- * ColumnarToRow (115) - : +- Scan parquet default.date_dim (114) - :- * Project (126) - : +- * BroadcastHashJoin Inner BuildRight (125) - : :- * Filter (123) - : : +- * ColumnarToRow (122) - : : +- Scan parquet default.catalog_sales (121) - : +- ReusedExchange (124) - +- * Project (132) - +- * BroadcastHashJoin Inner BuildRight (131) - :- * Filter (129) - : +- * ColumnarToRow (128) - : +- Scan parquet default.web_sales (127) - +- ReusedExchange (130) - - -(111) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 85 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (135) ++- Exchange (134) + +- * HashAggregate (133) + +- Union (132) + :- * Project (119) + : +- * BroadcastHashJoin Inner BuildRight (118) + : :- * Filter (112) + : : +- * ColumnarToRow (111) + : : +- Scan parquet default.store_sales (110) + : +- BroadcastExchange (117) + : +- * Project (116) + : +- * Filter (115) + : +- * ColumnarToRow (114) + : +- Scan parquet default.date_dim (113) + :- * Project (125) + : +- * BroadcastHashJoin Inner BuildRight (124) + : :- * Filter (122) + : : +- * ColumnarToRow (121) + : : +- Scan parquet default.catalog_sales (120) + : +- ReusedExchange (123) + +- * Project (131) + +- * BroadcastHashJoin Inner BuildRight (130) + :- * Filter (128) + : +- * ColumnarToRow (127) + : +- Scan parquet default.web_sales (126) + +- ReusedExchange (129) + + +(110) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(111) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(113) Filter [codegen id : 2] +(112) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(114) Scan parquet default.date_dim +(113) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(115) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(116) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(117) Project [codegen id : 1] +(116) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(118) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] -(119) BroadcastHashJoin [codegen id : 2] +(118) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(120) Project [codegen id : 2] +(119) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#72, ss_list_price#4 AS list_price#73] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(121) Scan parquet default.catalog_sales +(120) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] +(121) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] -(123) Filter [codegen id : 4] +(122) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Condition : isnotnull(cs_sold_date_sk#18) -(124) ReusedExchange [Reuses operator id: 118] +(123) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#10] -(125) BroadcastHashJoin [codegen id : 4] +(124) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(126) Project [codegen id : 4] +(125) Project [codegen id : 4] Output [2]: [cs_quantity#74 AS quantity#76, cs_list_price#75 AS list_price#77] Input [4]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75, d_date_sk#10] -(127) Scan parquet default.web_sales +(126) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 6] +(127) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] -(129) Filter [codegen id : 6] +(128) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Condition : isnotnull(ws_sold_date_sk#22) -(130) ReusedExchange [Reuses operator id: 118] +(129) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#10] -(131) BroadcastHashJoin [codegen id : 6] +(130) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(132) Project [codegen id : 6] +(131) Project [codegen id : 6] Output [2]: [ws_quantity#78 AS quantity#80, ws_list_price#79 AS list_price#81] Input [4]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79, d_date_sk#10] -(133) Union +(132) Union -(134) HashAggregate [codegen id : 7] +(133) HashAggregate [codegen id : 7] Input [2]: [quantity#72, list_price#73] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#82, count#83] Results [2]: [sum#84, count#85] -(135) Exchange +(134) Exchange Input [2]: [sum#84, count#85] Arguments: SinglePartition, true, [id=#86] -(136) HashAggregate [codegen id : 8] +(135) HashAggregate [codegen id : 8] Input [2]: [sum#84, count#85] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87 AS average_sales#88] -Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* Project (140) -+- * Filter (139) - +- * ColumnarToRow (138) - +- Scan parquet default.date_dim (137) +Subquery:2 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* Project (139) ++- * Filter (138) + +- * ColumnarToRow (137) + +- Scan parquet default.date_dim (136) -(137) Scan parquet default.date_dim +(136) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 1] +(137) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(139) Filter [codegen id : 1] +(138) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 2000)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11)) -(140) Project [codegen id : 1] +(139) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] +Subquery:3 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:4 Hosting operator id = 94 Hosting Expression = Subquery scalar-subquery#50, [id=#51] +* Project (143) ++- * Filter (142) + +- * ColumnarToRow (141) + +- Scan parquet default.date_dim (140) -(141) Scan parquet default.date_dim +(140) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(143) Filter [codegen id : 1] +(142) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11)) -(144) Project [codegen id : 1] +(143) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] 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 d6b8ba4395d2e..db4f9c33ea415 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 @@ -81,81 +81,80 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_item_sk,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] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (13) + Exchange [i_brand_id,i_class_id,i_category_id] #9 + WholeStageCodegen (9) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #12 WholeStageCodegen (19) 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 1f31ded51f1ef..38134e1d90073 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 @@ -1,104 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (99) ++- * BroadcastHashJoin Inner BuildRight (98) + :- * Project (76) + : +- * Filter (75) + : +- * HashAggregate (74) + : +- Exchange (73) + : +- * HashAggregate (72) + : +- * Project (71) + : +- * BroadcastHashJoin Inner BuildRight (70) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * BroadcastHashJoin LeftSemi BuildRight (56) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (55) + : : : +- * Project (54) + : : : +- * BroadcastHashJoin Inner BuildRight (53) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) + : : : +- BroadcastExchange (52) + : : : +- * HashAggregate (51) + : : : +- * HashAggregate (50) + : : : +- Exchange (49) + : : : +- * HashAggregate (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (39) + : : : : : +- * ColumnarToRow (38) + : : : : : +- Scan parquet default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (62) + : : +- * BroadcastHashJoin LeftSemi BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.item (57) + : : +- ReusedExchange (60) + : +- BroadcastExchange (69) + : +- * Project (68) + : +- * Filter (67) + : +- * ColumnarToRow (66) + : +- Scan parquet default.date_dim (65) + +- BroadcastExchange (97) + +- * Project (96) + +- * Filter (95) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * Project (91) + +- * BroadcastHashJoin Inner BuildRight (90) + :- * Project (84) + : +- * BroadcastHashJoin Inner BuildRight (83) + : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : :- * Filter (79) + : : : +- * ColumnarToRow (78) + : : : +- Scan parquet default.store_sales (77) + : : +- ReusedExchange (80) + : +- ReusedExchange (82) + +- BroadcastExchange (89) + +- * Project (88) + +- * Filter (87) + +- * ColumnarToRow (86) + +- Scan parquet default.date_dim (85) (1) Scan parquet default.store_sales @@ -335,428 +334,421 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(52) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(53) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(54) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(55) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(57) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(58) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(59) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) -(61) ReusedExchange [Reuses operator id: 56] +(60) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(61) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(62) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(65) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(66) ColumnarToRow [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] -(68) Filter [codegen id : 24] +(67) Filter [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#29, [id=#30])) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(68) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(70) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] -(71) BroadcastHashJoin [codegen id : 25] +(70) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(71) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(72) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] -(74) Exchange +(73) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#38] -(75) HashAggregate [codegen id : 52] +(74) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(76) Filter [codegen id : 52] +(75) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(77) Project [codegen id : 52] +(76) Project [codegen id : 52] Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(78) Scan parquet default.store_sales +(77) Scan parquet default.store_sales Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 50] +(78) ColumnarToRow [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(80) Filter [codegen id : 50] +(79) Filter [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(81) ReusedExchange [Reuses operator id: 56] +(80) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 50] +(81) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(82) ReusedExchange [Reuses operator id: 62] Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(84) BroadcastHashJoin [codegen id : 50] +(83) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#47] Join condition: None -(85) Project [codegen id : 50] +(84) Project [codegen id : 50] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(86) Scan parquet default.date_dim +(85) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 49] +(86) ColumnarToRow [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] -(88) Filter [codegen id : 49] +(87) Filter [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#10)) -(89) Project [codegen id : 49] +(88) Project [codegen id : 49] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(90) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(91) BroadcastHashJoin [codegen id : 50] +(90) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(92) Project [codegen id : 50] +(91) Project [codegen id : 50] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50, d_date_sk#10] -(93) HashAggregate [codegen id : 50] +(92) HashAggregate [codegen id : 50] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] -(94) Exchange +(93) Exchange Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), true, [id=#60] -(95) HashAggregate [codegen id : 51] +(94) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(96) Filter [codegen id : 51] +(95) Filter [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(97) Project [codegen id : 51] +(96) Project [codegen id : 51] Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(98) BroadcastExchange +(97) BroadcastExchange Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] -(99) BroadcastHashJoin [codegen id : 52] +(98) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Join condition: None -(100) TakeOrderedAndProject +(99) TakeOrderedAndProject Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * 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) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.catalog_sales (111) - : +- ReusedExchange (114) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * Filter (119) - : +- * ColumnarToRow (118) - : +- Scan parquet default.web_sales (117) - +- ReusedExchange (120) - - -(101) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 75 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* HashAggregate (125) ++- Exchange (124) + +- * HashAggregate (123) + +- Union (122) + :- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * Filter (102) + : : +- * ColumnarToRow (101) + : : +- Scan parquet default.store_sales (100) + : +- BroadcastExchange (107) + : +- * Project (106) + : +- * Filter (105) + : +- * ColumnarToRow (104) + : +- Scan parquet default.date_dim (103) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- * Filter (112) + : : +- * ColumnarToRow (111) + : : +- Scan parquet default.catalog_sales (110) + : +- ReusedExchange (113) + +- * Project (121) + +- * BroadcastHashJoin Inner BuildRight (120) + :- * Filter (118) + : +- * ColumnarToRow (117) + : +- Scan parquet default.web_sales (116) + +- ReusedExchange (119) + + +(100) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] +(101) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(103) Filter [codegen id : 2] +(102) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(104) Scan parquet default.date_dim +(103) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(104) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(106) Filter [codegen id : 1] +(105) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(107) Project [codegen id : 1] +(106) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(108) BroadcastExchange +(107) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] -(109) BroadcastHashJoin [codegen id : 2] +(108) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(110) Project [codegen id : 2] +(109) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#69, ss_list_price#4 AS list_price#70] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(111) Scan parquet default.catalog_sales +(110) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] +(111) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] -(113) Filter [codegen id : 4] +(112) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Condition : isnotnull(cs_sold_date_sk#16) -(114) ReusedExchange [Reuses operator id: 108] +(113) ReusedExchange [Reuses operator id: 107] Output [1]: [d_date_sk#10] -(115) BroadcastHashJoin [codegen id : 4] +(114) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(116) Project [codegen id : 4] +(115) Project [codegen id : 4] Output [2]: [cs_quantity#71 AS quantity#73, cs_list_price#72 AS list_price#74] Input [4]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72, d_date_sk#10] -(117) Scan parquet default.web_sales +(116) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 6] +(117) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] -(119) Filter [codegen id : 6] +(118) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Condition : isnotnull(ws_sold_date_sk#20) -(120) ReusedExchange [Reuses operator id: 108] +(119) ReusedExchange [Reuses operator id: 107] Output [1]: [d_date_sk#10] -(121) BroadcastHashJoin [codegen id : 6] +(120) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(122) Project [codegen id : 6] +(121) Project [codegen id : 6] Output [2]: [ws_quantity#75 AS quantity#77, ws_list_price#76 AS list_price#78] Input [4]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76, d_date_sk#10] -(123) Union +(122) Union -(124) HashAggregate [codegen id : 7] +(123) HashAggregate [codegen id : 7] Input [2]: [quantity#69, list_price#70] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#79, count#80] Results [2]: [sum#81, count#82] -(125) Exchange +(124) Exchange Input [2]: [sum#81, count#82] Arguments: SinglePartition, true, [id=#83] -(126) HashAggregate [codegen id : 8] +(125) HashAggregate [codegen id : 8] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS average_sales#85] -Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* Project (129) ++- * Filter (128) + +- * ColumnarToRow (127) + +- Scan parquet default.date_dim (126) -(127) Scan parquet default.date_dim +(126) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 1] +(127) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(129) Filter [codegen id : 1] +(128) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 2000)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) -(130) Project [codegen id : 1] +(129) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:3 Hosting operator id = 95 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (134) -+- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) +Subquery:4 Hosting operator id = 87 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* Project (133) ++- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet default.date_dim (130) -(131) Scan parquet default.date_dim +(130) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 1] +(131) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(133) Filter [codegen id : 1] +(132) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) -(134) Project [codegen id : 1] +(133) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] 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 7bbf83e3de707..9bb0d49edff50 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 @@ -73,71 +73,70 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #4 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #4 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_item_sk,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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) + BroadcastExchange #7 + WholeStageCodegen (5) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + BroadcastExchange #8 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (23) 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 85f71b6cd9388..e93dc9996cd84 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,103 +1,99 @@ == Physical Plan == -* HashAggregate (99) -+- Exchange (98) - +- * HashAggregate (97) - +- Union (96) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (52) - : : +- SortMergeJoin LeftSemi (51) - : : :- * Sort (33) - : : : +- Exchange (32) - : : : +- * Project (31) - : : : +- SortMergeJoin LeftSemi (30) +* HashAggregate (95) ++- Exchange (94) + +- * HashAggregate (93) + +- Union (92) + :- * Project (57) + : +- * BroadcastHashJoin Inner BuildRight (56) + : :- * Project (50) + : : +- SortMergeJoin LeftSemi (49) + : : :- * Sort (32) + : : : +- Exchange (31) + : : : +- * Project (30) + : : : +- SortMergeJoin LeftSemi (29) : : : :- * 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 (50) - : : +- * Project (49) - : : +- * Filter (48) - : : +- * HashAggregate (47) - : : +- * HashAggregate (46) - : : +- * Project (45) - : : +- * SortMergeJoin Inner (44) - : : :- * Sort (38) - : : : +- Exchange (37) - : : : +- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.store_sales (34) - : : +- * Sort (43) - : : +- Exchange (42) - : : +- * Filter (41) - : : +- * ColumnarToRow (40) - : : +- Scan parquet default.customer (39) - : +- BroadcastExchange (57) - : +- * Project (56) - : +- * Filter (55) - : +- * ColumnarToRow (54) - : +- Scan parquet default.date_dim (53) - +- * Project (95) - +- * BroadcastHashJoin Inner BuildRight (94) - :- * Project (92) - : +- SortMergeJoin LeftSemi (91) - : :- * Sort (79) - : : +- Exchange (78) - : : +- * Project (77) - : : +- SortMergeJoin LeftSemi (76) - : : :- * Sort (64) - : : : +- Exchange (63) - : : : +- * Filter (62) - : : : +- * ColumnarToRow (61) - : : : +- Scan parquet default.web_sales (60) - : : +- * Sort (75) - : : +- * Project (74) - : : +- * Filter (73) - : : +- * HashAggregate (72) - : : +- * HashAggregate (71) - : : +- * Project (70) - : : +- * SortMergeJoin Inner (69) - : : :- * Sort (66) - : : : +- ReusedExchange (65) - : : +- * Sort (68) - : : +- ReusedExchange (67) - : +- * Sort (90) - : +- * Project (89) - : +- * Filter (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (81) - : : +- ReusedExchange (80) - : +- * Sort (83) - : +- ReusedExchange (82) - +- ReusedExchange (93) + : : : +- * Sort (28) + : : : +- * Project (27) + : : : +- * Filter (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 (48) + : : +- * Project (47) + : : +- * Filter (46) + : : +- * HashAggregate (45) + : : +- * Project (44) + : : +- * SortMergeJoin Inner (43) + : : :- * Sort (37) + : : : +- Exchange (36) + : : : +- * Filter (35) + : : : +- * ColumnarToRow (34) + : : : +- Scan parquet default.store_sales (33) + : : +- * Sort (42) + : : +- Exchange (41) + : : +- * Filter (40) + : : +- * ColumnarToRow (39) + : : +- Scan parquet default.customer (38) + : +- BroadcastExchange (55) + : +- * Project (54) + : +- * Filter (53) + : +- * ColumnarToRow (52) + : +- Scan parquet default.date_dim (51) + +- * Project (91) + +- * BroadcastHashJoin Inner BuildRight (90) + :- * Project (88) + : +- SortMergeJoin LeftSemi (87) + : :- * Sort (76) + : : +- Exchange (75) + : : +- * Project (74) + : : +- SortMergeJoin LeftSemi (73) + : : :- * Sort (62) + : : : +- Exchange (61) + : : : +- * Filter (60) + : : : +- * ColumnarToRow (59) + : : : +- Scan parquet default.web_sales (58) + : : +- * Sort (72) + : : +- * Project (71) + : : +- * Filter (70) + : : +- * HashAggregate (69) + : : +- * Project (68) + : : +- * SortMergeJoin Inner (67) + : : :- * Sort (64) + : : : +- ReusedExchange (63) + : : +- * Sort (66) + : : +- ReusedExchange (65) + : +- * Sort (86) + : +- * Project (85) + : +- * Filter (84) + : +- * HashAggregate (83) + : +- * Project (82) + : +- * SortMergeJoin Inner (81) + : :- * Sort (78) + : : +- ReusedExchange (77) + : +- * Sort (80) + : +- ReusedExchange (79) + +- ReusedExchange (89) (1) Scan parquet default.catalog_sales @@ -209,488 +205,452 @@ Input [4]: [ss_item_sk#8, d_date#10, i_item_sk#14, i_item_desc#15] (25) HashAggregate [codegen id : 8] Input [3]: [d_date#10, i_item_sk#14, i_item_desc#15] Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#10] -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#10, count#19] - -(26) HashAggregate [codegen id : 8] -Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#10, count#19] -Keys [3]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#10] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS count(1)#22] +Aggregate Attributes [1]: [count(1)#18] +Results [2]: [i_item_sk#14 AS item_sk#19, count(1)#18 AS count(1)#20] -(27) Filter [codegen id : 8] -Input [2]: [item_sk#21, count(1)#22] -Condition : (count(1)#22 > 4) +(26) Filter [codegen id : 8] +Input [2]: [item_sk#19, count(1)#20] +Condition : (count(1)#20 > 4) -(28) Project [codegen id : 8] -Output [1]: [item_sk#21] -Input [2]: [item_sk#21, count(1)#22] +(27) Project [codegen id : 8] +Output [1]: [item_sk#19] +Input [2]: [item_sk#19, count(1)#20] -(29) Sort [codegen id : 8] -Input [1]: [item_sk#21] -Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 +(28) Sort [codegen id : 8] +Input [1]: [item_sk#19] +Arguments: [item_sk#19 ASC NULLS FIRST], false, 0 -(30) SortMergeJoin +(29) SortMergeJoin Left keys [1]: [cs_item_sk#3] -Right keys [1]: [item_sk#21] +Right keys [1]: [item_sk#19] Join condition: None -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -(32) Exchange +(31) Exchange Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#23] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] -(33) Sort [codegen id : 10] +(32) Sort [codegen id : 10] Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(33) Scan parquet default.store_sales +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 11] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(34) ColumnarToRow [codegen id : 11] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(36) Filter [codegen id : 11] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Condition : isnotnull(ss_customer_sk#24) +(35) Filter [codegen id : 11] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Condition : isnotnull(ss_customer_sk#22) -(37) Exchange -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#27] +(36) Exchange +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: hashpartitioning(ss_customer_sk#22, 5), true, [id=#25] -(38) Sort [codegen id : 12] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 12] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer -Output [1]: [c_customer_sk#28] +(38) Scan parquet default.customer +Output [1]: [c_customer_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#28] +(39) ColumnarToRow [codegen id : 13] +Input [1]: [c_customer_sk#26] -(41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#28] -Condition : isnotnull(c_customer_sk#28) +(40) Filter [codegen id : 13] +Input [1]: [c_customer_sk#26] +Condition : isnotnull(c_customer_sk#26) -(42) Exchange -Input [1]: [c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#29] +(41) Exchange +Input [1]: [c_customer_sk#26] +Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#27] -(43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(42) Sort [codegen id : 14] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 15] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(43) SortMergeJoin [codegen id : 15] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(45) Project [codegen id : 15] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#30, isEmpty#31] -Results [3]: [c_customer_sk#28, sum#32, isEmpty#33] - -(47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34] -Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] - -(48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) - -(49) Project [codegen id : 15] -Output [1]: [c_customer_sk#28] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] - -(50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin +(44) Project [codegen id : 15] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(45) HashAggregate [codegen id : 15] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#28] +Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#28 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] + +(46) Filter [codegen id : 15] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#30, [id=#31] as decimal(32,6)))), DecimalType(38,8), true))) + +(47) Project [codegen id : 15] +Output [1]: [c_customer_sk#26] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] + +(48) Sort [codegen id : 15] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 + +(49) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#26] Join condition: None -(52) Project [codegen id : 17] +(50) Project [codegen id : 17] Output [3]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5] Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -(53) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#11, d_moy#38] +(51) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#11, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#9, d_year#11, d_moy#38] +(52) ColumnarToRow [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#32] -(55) Filter [codegen id : 16] -Input [3]: [d_date_sk#9, d_year#11, d_moy#38] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#38)) AND (d_year#11 = 2000)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#9)) +(53) Filter [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#32] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#32)) AND (d_year#11 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#9)) -(56) Project [codegen id : 16] +(54) Project [codegen id : 16] Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#11, d_moy#38] +Input [3]: [d_date_sk#9, d_year#11, d_moy#32] -(57) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] -(58) BroadcastHashJoin [codegen id : 17] +(56) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#1] Right keys [1]: [d_date_sk#9] Join condition: None -(59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true) AS sales#40] +(57) Project [codegen id : 17] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true) AS sales#34] Input [4]: [cs_sold_date_sk#1, cs_quantity#4, cs_list_price#5, d_date_sk#9] -(60) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +(58) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#35, ws_item_sk#36, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +(59) ColumnarToRow [codegen id : 18] +Input [5]: [ws_sold_date_sk#35, ws_item_sk#36, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] -(62) Filter [codegen id : 18] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Condition : isnotnull(ws_sold_date_sk#41) +(60) Filter [codegen id : 18] +Input [5]: [ws_sold_date_sk#35, ws_item_sk#36, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] +Condition : isnotnull(ws_sold_date_sk#35) -(63) Exchange -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Arguments: hashpartitioning(ws_item_sk#42, 5), true, [id=#46] +(61) Exchange +Input [5]: [ws_sold_date_sk#35, ws_item_sk#36, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] +Arguments: hashpartitioning(ws_item_sk#36, 5), true, [id=#40] -(64) Sort [codegen id : 19] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Arguments: [ws_item_sk#42 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 19] +Input [5]: [ws_sold_date_sk#35, ws_item_sk#36, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] +Arguments: [ws_item_sk#36 ASC NULLS FIRST], false, 0 -(65) ReusedExchange [Reuses operator id: 16] +(63) ReusedExchange [Reuses operator id: 16] Output [2]: [ss_item_sk#8, d_date#10] -(66) Sort [codegen id : 22] +(64) Sort [codegen id : 22] Input [2]: [ss_item_sk#8, d_date#10] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(67) ReusedExchange [Reuses operator id: 21] +(65) ReusedExchange [Reuses operator id: 21] Output [2]: [i_item_sk#14, i_item_desc#15] -(68) Sort [codegen id : 24] +(66) Sort [codegen id : 24] Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 25] +(67) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#14] Join condition: None -(70) Project [codegen id : 25] +(68) Project [codegen id : 25] Output [3]: [d_date#10, i_item_sk#14, i_item_desc#15] Input [4]: [ss_item_sk#8, d_date#10, i_item_sk#14, i_item_desc#15] -(71) HashAggregate [codegen id : 25] +(69) HashAggregate [codegen id : 25] Input [3]: [d_date#10, i_item_sk#14, i_item_desc#15] -Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#48] -Results [4]: [substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10, count#49] - -(72) HashAggregate [codegen id : 25] -Input [4]: [substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10, count#49] -Keys [3]: [substr(i_item_desc#15, 1, 30)#47, i_item_sk#14, d_date#10] +Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#41, i_item_sk#14, d_date#10] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#50 AS count(1)#51] +Aggregate Attributes [1]: [count(1)#42] +Results [2]: [i_item_sk#14 AS item_sk#19, count(1)#42 AS count(1)#43] -(73) Filter [codegen id : 25] -Input [2]: [item_sk#21, count(1)#51] -Condition : (count(1)#51 > 4) +(70) Filter [codegen id : 25] +Input [2]: [item_sk#19, count(1)#43] +Condition : (count(1)#43 > 4) -(74) Project [codegen id : 25] -Output [1]: [item_sk#21] -Input [2]: [item_sk#21, count(1)#51] +(71) Project [codegen id : 25] +Output [1]: [item_sk#19] +Input [2]: [item_sk#19, count(1)#43] -(75) Sort [codegen id : 25] -Input [1]: [item_sk#21] -Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 +(72) Sort [codegen id : 25] +Input [1]: [item_sk#19] +Arguments: [item_sk#19 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [item_sk#21] +(73) SortMergeJoin +Left keys [1]: [ws_item_sk#36] +Right keys [1]: [item_sk#19] Join condition: None -(77) Project [codegen id : 26] -Output [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Input [5]: [ws_sold_date_sk#41, ws_item_sk#42, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +(74) Project [codegen id : 26] +Output [4]: [ws_sold_date_sk#35, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] +Input [5]: [ws_sold_date_sk#35, ws_item_sk#36, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] -(78) Exchange -Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Arguments: hashpartitioning(ws_bill_customer_sk#43, 5), true, [id=#52] +(75) Exchange +Input [4]: [ws_sold_date_sk#35, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] +Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), true, [id=#44] -(79) Sort [codegen id : 27] -Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] -Arguments: [ws_bill_customer_sk#43 ASC NULLS FIRST], false, 0 +(76) Sort [codegen id : 27] +Input [4]: [ws_sold_date_sk#35, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] +Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 -(80) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(77) ReusedExchange [Reuses operator id: 36] +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(81) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 29] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(82) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#28] +(79) ReusedExchange [Reuses operator id: 41] +Output [1]: [c_customer_sk#26] -(83) Sort [codegen id : 31] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(80) Sort [codegen id : 31] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(84) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(81) SortMergeJoin [codegen id : 32] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None +(82) Project [codegen id : 32] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(83) HashAggregate [codegen id : 32] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#45] +Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#45 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#46] + +(84) Filter [codegen id : 32] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#46] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#30, [id=#31] as decimal(32,6)))), DecimalType(38,8), true))) + (85) Project [codegen id : 32] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(86) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [3]: [c_customer_sk#28, sum#55, isEmpty#56] - -(87) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#28, sum#55, isEmpty#56] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#57] -Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58] - -(88) Filter [codegen id : 32] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) - -(89) Project [codegen id : 32] -Output [1]: [c_customer_sk#28] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#58] - -(90) Sort [codegen id : 32] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 - -(91) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#43] -Right keys [1]: [c_customer_sk#28] +Output [1]: [c_customer_sk#26] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#46] + +(86) Sort [codegen id : 32] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 + +(87) SortMergeJoin +Left keys [1]: [ws_bill_customer_sk#37] +Right keys [1]: [c_customer_sk#26] Join condition: None -(92) Project [codegen id : 34] -Output [3]: [ws_sold_date_sk#41, ws_quantity#44, ws_list_price#45] -Input [4]: [ws_sold_date_sk#41, ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45] +(88) Project [codegen id : 34] +Output [3]: [ws_sold_date_sk#35, ws_quantity#38, ws_list_price#39] +Input [4]: [ws_sold_date_sk#35, ws_bill_customer_sk#37, ws_quantity#38, ws_list_price#39] -(93) ReusedExchange [Reuses operator id: 57] +(89) ReusedExchange [Reuses operator id: 55] Output [1]: [d_date_sk#9] -(94) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#41] +(90) BroadcastHashJoin [codegen id : 34] +Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#9] Join condition: None -(95) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#44 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#45 as decimal(12,2)))), DecimalType(18,2), true) AS sales#59] -Input [4]: [ws_sold_date_sk#41, ws_quantity#44, ws_list_price#45, d_date_sk#9] +(91) Project [codegen id : 34] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#39 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [ws_sold_date_sk#35, ws_quantity#38, ws_list_price#39, d_date_sk#9] -(96) Union +(92) Union -(97) HashAggregate [codegen id : 35] -Input [1]: [sales#40] +(93) HashAggregate [codegen id : 35] +Input [1]: [sales#34] Keys: [] -Functions [1]: [partial_sum(sales#40)] -Aggregate Attributes [2]: [sum#60, isEmpty#61] -Results [2]: [sum#62, isEmpty#63] +Functions [1]: [partial_sum(sales#34)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [2]: [sum#50, isEmpty#51] -(98) Exchange -Input [2]: [sum#62, isEmpty#63] -Arguments: SinglePartition, true, [id=#64] +(94) Exchange +Input [2]: [sum#50, isEmpty#51] +Arguments: SinglePartition, true, [id=#52] -(99) HashAggregate [codegen id : 36] -Input [2]: [sum#62, isEmpty#63] +(95) HashAggregate [codegen id : 36] +Input [2]: [sum#50, isEmpty#51] Keys: [] -Functions [1]: [sum(sales#40)] -Aggregate Attributes [1]: [sum(sales#40)#65] -Results [1]: [sum(sales#40)#65 AS sum(sales)#66] +Functions [1]: [sum(sales#34)] +Aggregate Attributes [1]: [sum(sales#34)#53] +Results [1]: [sum(sales#34)#53 AS sum(sales)#54] ===== Subqueries ===== -Subquery:1 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#36, [id=#37] -* HashAggregate (123) -+- Exchange (122) - +- * HashAggregate (121) - +- * HashAggregate (120) - +- * HashAggregate (119) - +- * Project (118) - +- * SortMergeJoin Inner (117) - :- * Sort (111) - : +- Exchange (110) - : +- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * Filter (102) - : : +- * ColumnarToRow (101) - : : +- Scan parquet default.store_sales (100) - : +- BroadcastExchange (107) - : +- * Project (106) - : +- * Filter (105) - : +- * ColumnarToRow (104) - : +- Scan parquet default.date_dim (103) - +- * Sort (116) - +- Exchange (115) - +- * Filter (114) - +- * ColumnarToRow (113) - +- Scan parquet default.customer (112) - - -(100) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* HashAggregate (118) ++- Exchange (117) + +- * HashAggregate (116) + +- * HashAggregate (115) + +- * Project (114) + +- * SortMergeJoin Inner (113) + :- * Sort (107) + : +- Exchange (106) + : +- * Project (105) + : +- * BroadcastHashJoin Inner BuildRight (104) + : :- * Filter (98) + : : +- * ColumnarToRow (97) + : : +- Scan parquet default.store_sales (96) + : +- BroadcastExchange (103) + : +- * Project (102) + : +- * Filter (101) + : +- * ColumnarToRow (100) + : +- Scan parquet default.date_dim (99) + +- * Sort (112) + +- Exchange (111) + +- * Filter (110) + +- * ColumnarToRow (109) + +- Scan parquet default.customer (108) + + +(96) Scan parquet default.store_sales +Output [4]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(101) ColumnarToRow [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(97) ColumnarToRow [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(102) Filter [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Condition : (isnotnull(ss_customer_sk#24) AND isnotnull(ss_sold_date_sk#7)) +(98) Filter [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Condition : (isnotnull(ss_customer_sk#22) AND isnotnull(ss_sold_date_sk#7)) -(103) Scan parquet default.date_dim +(99) Scan parquet default.date_dim Output [2]: [d_date_sk#9, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 1] +(100) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] -(105) Filter [codegen id : 1] +(101) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] Condition : (d_year#11 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#9)) -(106) Project [codegen id : 1] +(102) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [2]: [d_date_sk#9, d_year#11] -(107) BroadcastExchange +(103) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] -(108) BroadcastHashJoin [codegen id : 2] +(104) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(109) Project [codegen id : 2] -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Input [5]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, d_date_sk#9] +(105) Project [codegen id : 2] +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Input [5]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, d_date_sk#9] -(110) Exchange -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#68] +(106) Exchange +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: hashpartitioning(ss_customer_sk#22, 5), true, [id=#56] -(111) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(107) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(112) Scan parquet default.customer -Output [1]: [c_customer_sk#28] +(108) Scan parquet default.customer +Output [1]: [c_customer_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(113) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#28] +(109) ColumnarToRow [codegen id : 4] +Input [1]: [c_customer_sk#26] -(114) Filter [codegen id : 4] -Input [1]: [c_customer_sk#28] -Condition : isnotnull(c_customer_sk#28) +(110) Filter [codegen id : 4] +Input [1]: [c_customer_sk#26] +Condition : isnotnull(c_customer_sk#26) -(115) Exchange -Input [1]: [c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#69] +(111) Exchange +Input [1]: [c_customer_sk#26] +Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#57] -(116) Sort [codegen id : 5] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(112) Sort [codegen id : 5] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(117) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(113) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(118) Project [codegen id : 6] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(119) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [3]: [c_customer_sk#28, sum#72, isEmpty#73] - -(120) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#28, sum#72, isEmpty#73] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#74] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#74 AS csales#75] - -(121) HashAggregate [codegen id : 6] -Input [1]: [csales#75] +(114) Project [codegen id : 6] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(115) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#58] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#58 AS csales#59] + +(116) HashAggregate [codegen id : 6] +Input [1]: [csales#59] Keys: [] -Functions [1]: [partial_max(csales#75)] -Aggregate Attributes [1]: [max#76] -Results [1]: [max#77] +Functions [1]: [partial_max(csales#59)] +Aggregate Attributes [1]: [max#60] +Results [1]: [max#61] -(122) Exchange -Input [1]: [max#77] -Arguments: SinglePartition, true, [id=#78] +(117) Exchange +Input [1]: [max#61] +Arguments: SinglePartition, true, [id=#62] -(123) HashAggregate [codegen id : 7] -Input [1]: [max#77] +(118) HashAggregate [codegen id : 7] +Input [1]: [max#61] Keys: [] -Functions [1]: [max(csales#75)] -Aggregate Attributes [1]: [max(csales#75)#79] -Results [1]: [max(csales#75)#79 AS tpcds_cmax#80] +Functions [1]: [max(csales#59)] +Aggregate Attributes [1]: [max(csales#59)#63] +Results [1]: [max(csales#59)#63 AS tpcds_cmax#64] -Subquery:2 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] +Subquery:2 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] 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 5bb8bc5b99d0c..a4600b94a9702 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 @@ -33,109 +33,106 @@ WholeStageCodegen (36) Sort [item_sk] Project [item_sk] Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (4) - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk,d_date] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - WholeStageCodegen (15) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - Subquery #1 - WholeStageCodegen (7) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #9 - WholeStageCodegen (6) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] + HashAggregate [i_item_desc,i_item_sk,d_date] [count(1),item_sk,count(1),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] + WholeStageCodegen (5) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #10 - WholeStageCodegen (2) - Project [ss_customer_sk,ss_quantity,ss_sales_price] + Exchange [ss_item_sk] #4 + WholeStageCodegen (4) + Project [ss_item_sk,d_date] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Project [d_date_sk] + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk,d_date] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_year] InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] + WholeStageCodegen (7) + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #12 - WholeStageCodegen (4) - Filter [c_customer_sk] + Exchange [i_item_sk] #6 + WholeStageCodegen (6) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #7 - WholeStageCodegen (11) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] - InputAdapter - WholeStageCodegen (14) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (13) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk] + Scan parquet default.item [i_item_sk,i_item_desc] + WholeStageCodegen (15) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + Subquery #1 + WholeStageCodegen (7) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #9 + WholeStageCodegen (6) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #10 + WholeStageCodegen (2) + Project [ss_customer_sk,ss_quantity,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #12 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #7 + WholeStageCodegen (11) + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] + InputAdapter + WholeStageCodegen (14) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (13) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk] InputAdapter BroadcastExchange #13 WholeStageCodegen (16) @@ -171,38 +168,36 @@ WholeStageCodegen (36) Sort [item_sk] Project [item_sk] Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (22) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk,d_date] #4 - InputAdapter - WholeStageCodegen (24) - Sort [i_item_sk] - InputAdapter - ReusedExchange [i_item_sk,i_item_desc] #6 + HashAggregate [i_item_desc,i_item_sk,d_date] [count(1),item_sk,count(1),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (22) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,d_date] #4 + InputAdapter + WholeStageCodegen (24) + Sort [i_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_desc] #6 WholeStageCodegen (32) Sort [c_customer_sk] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (29) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 - InputAdapter - WholeStageCodegen (31) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #8 + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (29) + Sort [ss_customer_sk] + InputAdapter + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 + InputAdapter + WholeStageCodegen (31) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk] #8 InputAdapter ReusedExchange [d_date_sk] #13 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index 9a4c2b064d091..90022594e9594 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -1,140 +1,134 @@ == Physical Plan == -TakeOrderedAndProject (136) -+- Union (135) - :- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * Project (77) - : +- * SortMergeJoin Inner (76) - : :- * Project (58) - : : +- * BroadcastHashJoin Inner BuildRight (57) - : : :- SortMergeJoin LeftSemi (51) - : : : :- * Sort (33) - : : : : +- Exchange (32) - : : : : +- * Project (31) - : : : : +- SortMergeJoin LeftSemi (30) +TakeOrderedAndProject (130) ++- Union (129) + :- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- * Project (74) + : +- * SortMergeJoin Inner (73) + : :- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- SortMergeJoin LeftSemi (49) + : : : :- * Sort (32) + : : : : +- Exchange (31) + : : : : +- * Project (30) + : : : : +- SortMergeJoin LeftSemi (29) : : : : :- * 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 (50) - : : : +- * Project (49) - : : : +- * Filter (48) - : : : +- * HashAggregate (47) - : : : +- * HashAggregate (46) - : : : +- * Project (45) - : : : +- * SortMergeJoin Inner (44) - : : : :- * Sort (38) - : : : : +- Exchange (37) - : : : : +- * Filter (36) - : : : : +- * ColumnarToRow (35) - : : : : +- Scan parquet default.store_sales (34) - : : : +- * Sort (43) - : : : +- Exchange (42) - : : : +- * Filter (41) - : : : +- * ColumnarToRow (40) - : : : +- Scan parquet default.customer (39) - : : +- BroadcastExchange (56) - : : +- * Project (55) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet default.date_dim (52) - : +- SortMergeJoin LeftSemi (75) - : :- * Sort (63) - : : +- Exchange (62) - : : +- * Filter (61) - : : +- * ColumnarToRow (60) - : : +- Scan parquet default.customer (59) - : +- * Sort (74) - : +- * Project (73) - : +- * Filter (72) - : +- * HashAggregate (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * SortMergeJoin Inner (68) - : :- * Sort (65) - : : +- ReusedExchange (64) - : +- * Sort (67) - : +- ReusedExchange (66) - +- * HashAggregate (134) - +- Exchange (133) - +- * HashAggregate (132) - +- * Project (131) - +- * SortMergeJoin Inner (130) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- SortMergeJoin LeftSemi (112) - : : :- * Sort (100) - : : : +- Exchange (99) - : : : +- * Project (98) - : : : +- SortMergeJoin LeftSemi (97) - : : : :- * Sort (85) - : : : : +- Exchange (84) - : : : : +- * Filter (83) - : : : : +- * ColumnarToRow (82) - : : : : +- Scan parquet default.web_sales (81) - : : : +- * Sort (96) - : : : +- * Project (95) - : : : +- * Filter (94) - : : : +- * HashAggregate (93) - : : : +- * HashAggregate (92) - : : : +- * Project (91) - : : : +- * SortMergeJoin Inner (90) - : : : :- * Sort (87) - : : : : +- ReusedExchange (86) - : : : +- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (111) - : : +- * Project (110) - : : +- * Filter (109) - : : +- * HashAggregate (108) - : : +- * HashAggregate (107) - : : +- * Project (106) - : : +- * SortMergeJoin Inner (105) - : : :- * Sort (102) - : : : +- ReusedExchange (101) - : : +- * Sort (104) - : : +- ReusedExchange (103) - : +- ReusedExchange (113) - +- SortMergeJoin LeftSemi (129) - :- * Sort (117) - : +- ReusedExchange (116) - +- * Sort (128) - +- * Project (127) - +- * Filter (126) - +- * HashAggregate (125) - +- * HashAggregate (124) - +- * Project (123) - +- * SortMergeJoin Inner (122) - :- * Sort (119) - : +- ReusedExchange (118) - +- * Sort (121) - +- ReusedExchange (120) + : : : : +- * Sort (28) + : : : : +- * Project (27) + : : : : +- * Filter (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 (48) + : : : +- * Project (47) + : : : +- * Filter (46) + : : : +- * HashAggregate (45) + : : : +- * Project (44) + : : : +- * SortMergeJoin Inner (43) + : : : :- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.store_sales (33) + : : : +- * Sort (42) + : : : +- Exchange (41) + : : : +- * Filter (40) + : : : +- * ColumnarToRow (39) + : : : +- Scan parquet default.customer (38) + : : +- BroadcastExchange (54) + : : +- * Project (53) + : : +- * Filter (52) + : : +- * ColumnarToRow (51) + : : +- Scan parquet default.date_dim (50) + : +- SortMergeJoin LeftSemi (72) + : :- * Sort (61) + : : +- Exchange (60) + : : +- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.customer (57) + : +- * Sort (71) + : +- * Project (70) + : +- * Filter (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * SortMergeJoin Inner (66) + : :- * Sort (63) + : : +- ReusedExchange (62) + : +- * Sort (65) + : +- ReusedExchange (64) + +- * HashAggregate (128) + +- Exchange (127) + +- * HashAggregate (126) + +- * Project (125) + +- * SortMergeJoin Inner (124) + :- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- SortMergeJoin LeftSemi (107) + : : :- * Sort (96) + : : : +- Exchange (95) + : : : +- * Project (94) + : : : +- SortMergeJoin LeftSemi (93) + : : : :- * Sort (82) + : : : : +- Exchange (81) + : : : : +- * Filter (80) + : : : : +- * ColumnarToRow (79) + : : : : +- Scan parquet default.web_sales (78) + : : : +- * Sort (92) + : : : +- * Project (91) + : : : +- * Filter (90) + : : : +- * HashAggregate (89) + : : : +- * Project (88) + : : : +- * SortMergeJoin Inner (87) + : : : :- * Sort (84) + : : : : +- ReusedExchange (83) + : : : +- * Sort (86) + : : : +- ReusedExchange (85) + : : +- * Sort (106) + : : +- * Project (105) + : : +- * Filter (104) + : : +- * HashAggregate (103) + : : +- * Project (102) + : : +- * SortMergeJoin Inner (101) + : : :- * Sort (98) + : : : +- ReusedExchange (97) + : : +- * Sort (100) + : : +- ReusedExchange (99) + : +- ReusedExchange (108) + +- SortMergeJoin LeftSemi (123) + :- * Sort (112) + : +- ReusedExchange (111) + +- * Sort (122) + +- * Project (121) + +- * Filter (120) + +- * HashAggregate (119) + +- * Project (118) + +- * SortMergeJoin Inner (117) + :- * Sort (114) + : +- ReusedExchange (113) + +- * Sort (116) + +- ReusedExchange (115) (1) Scan parquet default.catalog_sales @@ -246,661 +240,611 @@ Input [4]: [ss_item_sk#8, d_date#10, i_item_sk#14, i_item_desc#15] (25) HashAggregate [codegen id : 8] Input [3]: [d_date#10, i_item_sk#14, i_item_desc#15] Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#10] -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#10, count#19] - -(26) HashAggregate [codegen id : 8] -Input [4]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#10, count#19] -Keys [3]: [substr(i_item_desc#15, 1, 30)#17, i_item_sk#14, d_date#10] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS count(1)#22] +Aggregate Attributes [1]: [count(1)#18] +Results [2]: [i_item_sk#14 AS item_sk#19, count(1)#18 AS count(1)#20] -(27) Filter [codegen id : 8] -Input [2]: [item_sk#21, count(1)#22] -Condition : (count(1)#22 > 4) +(26) Filter [codegen id : 8] +Input [2]: [item_sk#19, count(1)#20] +Condition : (count(1)#20 > 4) -(28) Project [codegen id : 8] -Output [1]: [item_sk#21] -Input [2]: [item_sk#21, count(1)#22] +(27) Project [codegen id : 8] +Output [1]: [item_sk#19] +Input [2]: [item_sk#19, count(1)#20] -(29) Sort [codegen id : 8] -Input [1]: [item_sk#21] -Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 +(28) Sort [codegen id : 8] +Input [1]: [item_sk#19] +Arguments: [item_sk#19 ASC NULLS FIRST], false, 0 -(30) SortMergeJoin +(29) SortMergeJoin Left keys [1]: [cs_item_sk#3] -Right keys [1]: [item_sk#21] +Right keys [1]: [item_sk#19] Join condition: None -(31) Project [codegen id : 9] +(30) Project [codegen id : 9] Output [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5] -(32) Exchange +(31) Exchange Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#23] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] -(33) Sort [codegen id : 10] +(32) Sort [codegen id : 10] Input [4]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(33) Scan parquet default.store_sales +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 11] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(34) ColumnarToRow [codegen id : 11] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(36) Filter [codegen id : 11] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Condition : isnotnull(ss_customer_sk#24) +(35) Filter [codegen id : 11] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Condition : isnotnull(ss_customer_sk#22) -(37) Exchange -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#27] +(36) Exchange +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: hashpartitioning(ss_customer_sk#22, 5), true, [id=#25] -(38) Sort [codegen id : 12] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 12] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(39) Scan parquet default.customer -Output [1]: [c_customer_sk#28] +(38) Scan parquet default.customer +Output [1]: [c_customer_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#28] +(39) ColumnarToRow [codegen id : 13] +Input [1]: [c_customer_sk#26] -(41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#28] -Condition : isnotnull(c_customer_sk#28) +(40) Filter [codegen id : 13] +Input [1]: [c_customer_sk#26] +Condition : isnotnull(c_customer_sk#26) -(42) Exchange -Input [1]: [c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#29] +(41) Exchange +Input [1]: [c_customer_sk#26] +Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#27] -(43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(42) Sort [codegen id : 14] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 15] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(43) SortMergeJoin [codegen id : 15] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(45) Project [codegen id : 15] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#30, isEmpty#31] -Results [3]: [c_customer_sk#28, sum#32, isEmpty#33] - -(47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34] -Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] - -(48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) - -(49) Project [codegen id : 15] -Output [1]: [c_customer_sk#28] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] - -(50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin +(44) Project [codegen id : 15] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(45) HashAggregate [codegen id : 15] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#28] +Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#28 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] + +(46) Filter [codegen id : 15] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#30, [id=#31] as decimal(32,6)))), DecimalType(38,8), true))) + +(47) Project [codegen id : 15] +Output [1]: [c_customer_sk#26] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] + +(48) Sort [codegen id : 15] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 + +(49) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#26] Join condition: None -(52) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#11, d_moy#38] +(50) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#11, d_moy#32] 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 -(53) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#9, d_year#11, d_moy#38] +(51) ColumnarToRow [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#32] -(54) Filter [codegen id : 16] -Input [3]: [d_date_sk#9, d_year#11, d_moy#38] -Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#38)) AND (d_year#11 = 2000)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#9)) +(52) Filter [codegen id : 16] +Input [3]: [d_date_sk#9, d_year#11, d_moy#32] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#32)) AND (d_year#11 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#9)) -(55) Project [codegen id : 16] +(53) Project [codegen id : 16] Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#11, d_moy#38] +Input [3]: [d_date_sk#9, d_year#11, d_moy#32] -(56) BroadcastExchange +(54) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] -(57) BroadcastHashJoin [codegen id : 17] +(55) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#1] Right keys [1]: [d_date_sk#9] Join condition: None -(58) Project [codegen id : 17] +(56) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5] Input [5]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, d_date_sk#9] -(59) Scan parquet default.customer -Output [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +(57) Scan parquet default.customer +Output [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +(58) ColumnarToRow [codegen id : 18] +Input [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] -(61) Filter [codegen id : 18] -Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] -Condition : isnotnull(c_customer_sk#28) +(59) Filter [codegen id : 18] +Input [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] +Condition : isnotnull(c_customer_sk#26) -(62) Exchange -Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] -Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#42] +(60) Exchange +Input [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] +Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#36] -(63) Sort [codegen id : 19] -Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 19] +Input [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(64) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(62) ReusedExchange [Reuses operator id: 36] +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(65) Sort [codegen id : 21] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(63) Sort [codegen id : 21] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#28] +(64) ReusedExchange [Reuses operator id: 41] +Output [1]: [c_customer_sk#26] -(67) Sort [codegen id : 23] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(65) Sort [codegen id : 23] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(68) SortMergeJoin [codegen id : 24] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(66) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(69) Project [codegen id : 24] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(70) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#30, isEmpty#31] -Results [3]: [c_customer_sk#28, sum#32, isEmpty#33] - -(71) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34] -Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#34 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] - -(72) Filter [codegen id : 24] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) - -(73) Project [codegen id : 24] -Output [1]: [c_customer_sk#28 AS c_customer_sk#28#43] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] - -(74) Sort [codegen id : 24] -Input [1]: [c_customer_sk#28#43] -Arguments: [c_customer_sk#28#43 ASC NULLS FIRST], false, 0 - -(75) SortMergeJoin -Left keys [1]: [c_customer_sk#28] -Right keys [1]: [c_customer_sk#28#43] +(67) Project [codegen id : 24] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(68) HashAggregate [codegen id : 24] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#28] +Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#28 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] + +(69) Filter [codegen id : 24] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#30, [id=#31] as decimal(32,6)))), DecimalType(38,8), true))) + +(70) Project [codegen id : 24] +Output [1]: [c_customer_sk#26 AS c_customer_sk#26#37] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#29] + +(71) Sort [codegen id : 24] +Input [1]: [c_customer_sk#26#37] +Arguments: [c_customer_sk#26#37 ASC NULLS FIRST], false, 0 + +(72) SortMergeJoin +Left keys [1]: [c_customer_sk#26] +Right keys [1]: [c_customer_sk#26#37] Join condition: None -(76) SortMergeJoin [codegen id : 25] +(73) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#26] Join condition: None -(77) Project [codegen id : 25] -Output [4]: [cs_quantity#4, cs_list_price#5, c_first_name#40, c_last_name#41] -Input [6]: [cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, c_customer_sk#28, c_first_name#40, c_last_name#41] +(74) Project [codegen id : 25] +Output [4]: [cs_quantity#4, cs_list_price#5, c_first_name#34, c_last_name#35] +Input [6]: [cs_bill_customer_sk#2, cs_quantity#4, cs_list_price#5, c_customer_sk#26, c_first_name#34, c_last_name#35] -(78) HashAggregate [codegen id : 25] -Input [4]: [cs_quantity#4, cs_list_price#5, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] +(75) HashAggregate [codegen id : 25] +Input [4]: [cs_quantity#4, cs_list_price#5, c_first_name#34, c_last_name#35] +Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [4]: [c_last_name#41, c_first_name#40, sum#46, isEmpty#47] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [4]: [c_last_name#35, c_first_name#34, sum#40, isEmpty#41] -(79) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#46, isEmpty#47] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), true, [id=#48] +(76) Exchange +Input [4]: [c_last_name#35, c_first_name#34, sum#40, isEmpty#41] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), true, [id=#42] -(80) HashAggregate [codegen id : 26] -Input [4]: [c_last_name#41, c_first_name#40, sum#46, isEmpty#47] -Keys [2]: [c_last_name#41, c_first_name#40] +(77) HashAggregate [codegen id : 26] +Input [4]: [c_last_name#35, c_first_name#34, sum#40, isEmpty#41] +Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#49] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#49 AS sales#50] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#43] +Results [3]: [c_last_name#35, c_first_name#34, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#4 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#5 as decimal(12,2)))), DecimalType(18,2), true))#43 AS sales#44] -(81) Scan parquet default.web_sales -Output [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +(78) Scan parquet default.web_sales +Output [5]: [ws_sold_date_sk#45, ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 27] -Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +(79) ColumnarToRow [codegen id : 27] +Input [5]: [ws_sold_date_sk#45, ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] -(83) Filter [codegen id : 27] -Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Condition : (isnotnull(ws_bill_customer_sk#53) AND isnotnull(ws_sold_date_sk#51)) +(80) Filter [codegen id : 27] +Input [5]: [ws_sold_date_sk#45, ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Condition : (isnotnull(ws_bill_customer_sk#47) AND isnotnull(ws_sold_date_sk#45)) -(84) Exchange -Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Arguments: hashpartitioning(ws_item_sk#52, 5), true, [id=#56] +(81) Exchange +Input [5]: [ws_sold_date_sk#45, ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Arguments: hashpartitioning(ws_item_sk#46, 5), true, [id=#50] -(85) Sort [codegen id : 28] -Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Arguments: [ws_item_sk#52 ASC NULLS FIRST], false, 0 +(82) Sort [codegen id : 28] +Input [5]: [ws_sold_date_sk#45, ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Arguments: [ws_item_sk#46 ASC NULLS FIRST], false, 0 -(86) ReusedExchange [Reuses operator id: 16] +(83) ReusedExchange [Reuses operator id: 16] Output [2]: [ss_item_sk#8, d_date#10] -(87) Sort [codegen id : 31] +(84) Sort [codegen id : 31] Input [2]: [ss_item_sk#8, d_date#10] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(88) ReusedExchange [Reuses operator id: 21] +(85) ReusedExchange [Reuses operator id: 21] Output [2]: [i_item_sk#14, i_item_desc#15] -(89) Sort [codegen id : 33] +(86) Sort [codegen id : 33] Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 34] +(87) SortMergeJoin [codegen id : 34] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#14] Join condition: None -(91) Project [codegen id : 34] +(88) Project [codegen id : 34] Output [3]: [d_date#10, i_item_sk#14, i_item_desc#15] Input [4]: [ss_item_sk#8, d_date#10, i_item_sk#14, i_item_desc#15] -(92) HashAggregate [codegen id : 34] +(89) HashAggregate [codegen id : 34] Input [3]: [d_date#10, i_item_sk#14, i_item_desc#15] -Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#58] -Results [4]: [substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10, count#59] - -(93) HashAggregate [codegen id : 34] -Input [4]: [substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10, count#59] -Keys [3]: [substr(i_item_desc#15, 1, 30)#57, i_item_sk#14, d_date#10] +Keys [3]: [substr(i_item_desc#15, 1, 30) AS substr(i_item_desc#15, 1, 30)#51, i_item_sk#14, d_date#10] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#60 AS count(1)#61] +Aggregate Attributes [1]: [count(1)#52] +Results [2]: [i_item_sk#14 AS item_sk#19, count(1)#52 AS count(1)#53] -(94) Filter [codegen id : 34] -Input [2]: [item_sk#21, count(1)#61] -Condition : (count(1)#61 > 4) +(90) Filter [codegen id : 34] +Input [2]: [item_sk#19, count(1)#53] +Condition : (count(1)#53 > 4) -(95) Project [codegen id : 34] -Output [1]: [item_sk#21] -Input [2]: [item_sk#21, count(1)#61] +(91) Project [codegen id : 34] +Output [1]: [item_sk#19] +Input [2]: [item_sk#19, count(1)#53] -(96) Sort [codegen id : 34] -Input [1]: [item_sk#21] -Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 +(92) Sort [codegen id : 34] +Input [1]: [item_sk#19] +Arguments: [item_sk#19 ASC NULLS FIRST], false, 0 -(97) SortMergeJoin -Left keys [1]: [ws_item_sk#52] -Right keys [1]: [item_sk#21] +(93) SortMergeJoin +Left keys [1]: [ws_item_sk#46] +Right keys [1]: [item_sk#19] Join condition: None -(98) Project [codegen id : 35] -Output [4]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Input [5]: [ws_sold_date_sk#51, ws_item_sk#52, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] +(94) Project [codegen id : 35] +Output [4]: [ws_sold_date_sk#45, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Input [5]: [ws_sold_date_sk#45, ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] -(99) Exchange -Input [4]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), true, [id=#62] +(95) Exchange +Input [4]: [ws_sold_date_sk#45, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Arguments: hashpartitioning(ws_bill_customer_sk#47, 5), true, [id=#54] -(100) Sort [codegen id : 36] -Input [4]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 +(96) Sort [codegen id : 36] +Input [4]: [ws_sold_date_sk#45, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Arguments: [ws_bill_customer_sk#47 ASC NULLS FIRST], false, 0 -(101) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(97) ReusedExchange [Reuses operator id: 36] +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(102) Sort [codegen id : 38] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(98) Sort [codegen id : 38] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(103) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#28] +(99) ReusedExchange [Reuses operator id: 41] +Output [1]: [c_customer_sk#26] -(104) Sort [codegen id : 40] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(100) Sort [codegen id : 40] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(105) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(101) SortMergeJoin [codegen id : 41] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(106) Project [codegen id : 41] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(107) HashAggregate [codegen id : 41] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [3]: [c_customer_sk#28, sum#65, isEmpty#66] - -(108) HashAggregate [codegen id : 41] -Input [3]: [c_customer_sk#28, sum#65, isEmpty#66] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67] -Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] - -(109) Filter [codegen id : 41] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) - -(110) Project [codegen id : 41] -Output [1]: [c_customer_sk#28] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] - -(111) Sort [codegen id : 41] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 - -(112) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#53] -Right keys [1]: [c_customer_sk#28] +(102) Project [codegen id : 41] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(103) HashAggregate [codegen id : 41] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#55] +Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56] + +(104) Filter [codegen id : 41] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#30, [id=#31] as decimal(32,6)))), DecimalType(38,8), true))) + +(105) Project [codegen id : 41] +Output [1]: [c_customer_sk#26] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56] + +(106) Sort [codegen id : 41] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 + +(107) SortMergeJoin +Left keys [1]: [ws_bill_customer_sk#47] +Right keys [1]: [c_customer_sk#26] Join condition: None -(113) ReusedExchange [Reuses operator id: 56] +(108) ReusedExchange [Reuses operator id: 54] Output [1]: [d_date_sk#9] -(114) BroadcastHashJoin [codegen id : 43] -Left keys [1]: [ws_sold_date_sk#51] +(109) BroadcastHashJoin [codegen id : 43] +Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#9] Join condition: None -(115) Project [codegen id : 43] -Output [3]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55] -Input [5]: [ws_sold_date_sk#51, ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, d_date_sk#9] +(110) Project [codegen id : 43] +Output [3]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49] +Input [5]: [ws_sold_date_sk#45, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, d_date_sk#9] -(116) ReusedExchange [Reuses operator id: 62] -Output [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] +(111) ReusedExchange [Reuses operator id: 60] +Output [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] -(117) Sort [codegen id : 45] -Input [3]: [c_customer_sk#28, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(112) Sort [codegen id : 45] +Input [3]: [c_customer_sk#26, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(118) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(113) ReusedExchange [Reuses operator id: 36] +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(119) Sort [codegen id : 47] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 47] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(120) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#28] +(115) ReusedExchange [Reuses operator id: 41] +Output [1]: [c_customer_sk#26] -(121) Sort [codegen id : 49] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 49] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(122) SortMergeJoin [codegen id : 50] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(117) SortMergeJoin [codegen id : 50] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(123) Project [codegen id : 50] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(124) HashAggregate [codegen id : 50] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [3]: [c_customer_sk#28, sum#65, isEmpty#66] - -(125) HashAggregate [codegen id : 50] -Input [3]: [c_customer_sk#28, sum#65, isEmpty#66] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67] -Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] - -(126) Filter [codegen id : 50] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#36, [id=#37] as decimal(32,6)))), DecimalType(38,8), true))) - -(127) Project [codegen id : 50] -Output [1]: [c_customer_sk#28 AS c_customer_sk#28#69] -Input [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#68] - -(128) Sort [codegen id : 50] -Input [1]: [c_customer_sk#28#69] -Arguments: [c_customer_sk#28#69 ASC NULLS FIRST], false, 0 - -(129) SortMergeJoin -Left keys [1]: [c_customer_sk#28] -Right keys [1]: [c_customer_sk#28#69] +(118) Project [codegen id : 50] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(119) HashAggregate [codegen id : 50] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#55] +Results [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56] + +(120) Filter [codegen id : 50] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#30, [id=#31] as decimal(32,6)))), DecimalType(38,8), true))) + +(121) Project [codegen id : 50] +Output [1]: [c_customer_sk#26 AS c_customer_sk#26#57] +Input [2]: [c_customer_sk#26, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#56] + +(122) Sort [codegen id : 50] +Input [1]: [c_customer_sk#26#57] +Arguments: [c_customer_sk#26#57 ASC NULLS FIRST], false, 0 + +(123) SortMergeJoin +Left keys [1]: [c_customer_sk#26] +Right keys [1]: [c_customer_sk#26#57] Join condition: None -(130) SortMergeJoin [codegen id : 51] -Left keys [1]: [ws_bill_customer_sk#53] -Right keys [1]: [c_customer_sk#28] +(124) SortMergeJoin [codegen id : 51] +Left keys [1]: [ws_bill_customer_sk#47] +Right keys [1]: [c_customer_sk#26] Join condition: None -(131) Project [codegen id : 51] -Output [4]: [ws_quantity#54, ws_list_price#55, c_first_name#40, c_last_name#41] -Input [6]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, c_customer_sk#28, c_first_name#40, c_last_name#41] +(125) Project [codegen id : 51] +Output [4]: [ws_quantity#48, ws_list_price#49, c_first_name#34, c_last_name#35] +Input [6]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, c_customer_sk#26, c_first_name#34, c_last_name#35] -(132) HashAggregate [codegen id : 51] -Input [4]: [ws_quantity#54, ws_list_price#55, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [4]: [c_last_name#41, c_first_name#40, sum#72, isEmpty#73] +(126) HashAggregate [codegen id : 51] +Input [4]: [ws_quantity#48, ws_list_price#49, c_first_name#34, c_last_name#35] +Keys [2]: [c_last_name#35, c_first_name#34] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [4]: [c_last_name#35, c_first_name#34, sum#60, isEmpty#61] -(133) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#72, isEmpty#73] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), true, [id=#74] +(127) Exchange +Input [4]: [c_last_name#35, c_first_name#34, sum#60, isEmpty#61] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), true, [id=#62] -(134) HashAggregate [codegen id : 52] -Input [4]: [c_last_name#41, c_first_name#40, sum#72, isEmpty#73] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#75] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#75 AS sales#76] +(128) HashAggregate [codegen id : 52] +Input [4]: [c_last_name#35, c_first_name#34, sum#60, isEmpty#61] +Keys [2]: [c_last_name#35, c_first_name#34] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63] +Results [3]: [c_last_name#35, c_first_name#34, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#63 AS sales#64] -(135) Union +(129) Union -(136) TakeOrderedAndProject -Input [3]: [c_last_name#41, c_first_name#40, sales#50] -Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#50] +(130) TakeOrderedAndProject +Input [3]: [c_last_name#35, c_first_name#34, sales#44] +Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#44 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#44] ===== Subqueries ===== -Subquery:1 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#36, [id=#37] -* HashAggregate (160) -+- Exchange (159) - +- * HashAggregate (158) - +- * HashAggregate (157) - +- * HashAggregate (156) - +- * Project (155) - +- * SortMergeJoin Inner (154) - :- * Sort (148) - : +- Exchange (147) - : +- * Project (146) - : +- * BroadcastHashJoin Inner BuildRight (145) - : :- * Filter (139) - : : +- * ColumnarToRow (138) - : : +- Scan parquet default.store_sales (137) - : +- BroadcastExchange (144) - : +- * Project (143) - : +- * Filter (142) - : +- * ColumnarToRow (141) - : +- Scan parquet default.date_dim (140) - +- * Sort (153) - +- Exchange (152) - +- * Filter (151) - +- * ColumnarToRow (150) - +- Scan parquet default.customer (149) - - -(137) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* HashAggregate (153) ++- Exchange (152) + +- * HashAggregate (151) + +- * HashAggregate (150) + +- * Project (149) + +- * SortMergeJoin Inner (148) + :- * Sort (142) + : +- Exchange (141) + : +- * Project (140) + : +- * BroadcastHashJoin Inner BuildRight (139) + : :- * Filter (133) + : : +- * ColumnarToRow (132) + : : +- Scan parquet default.store_sales (131) + : +- BroadcastExchange (138) + : +- * Project (137) + : +- * Filter (136) + : +- * ColumnarToRow (135) + : +- Scan parquet default.date_dim (134) + +- * Sort (147) + +- Exchange (146) + +- * Filter (145) + +- * ColumnarToRow (144) + +- Scan parquet default.customer (143) + + +(131) Scan parquet default.store_sales +Output [4]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +(132) ColumnarToRow [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] -(139) Filter [codegen id : 2] -Input [4]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Condition : (isnotnull(ss_customer_sk#24) AND isnotnull(ss_sold_date_sk#7)) +(133) Filter [codegen id : 2] +Input [4]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Condition : (isnotnull(ss_customer_sk#22) AND isnotnull(ss_sold_date_sk#7)) -(140) Scan parquet default.date_dim +(134) Scan parquet default.date_dim Output [2]: [d_date_sk#9, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(141) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] -(142) Filter [codegen id : 1] +(136) Filter [codegen id : 1] Input [2]: [d_date_sk#9, d_year#11] Condition : (d_year#11 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#9)) -(143) Project [codegen id : 1] +(137) Project [codegen id : 1] Output [1]: [d_date_sk#9] Input [2]: [d_date_sk#9, d_year#11] -(144) BroadcastExchange +(138) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] -(145) BroadcastHashJoin [codegen id : 2] +(139) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join condition: None -(146) Project [codegen id : 2] -Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Input [5]: [ss_sold_date_sk#7, ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, d_date_sk#9] +(140) Project [codegen id : 2] +Output [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Input [5]: [ss_sold_date_sk#7, ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, d_date_sk#9] -(147) Exchange -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: hashpartitioning(ss_customer_sk#24, 5), true, [id=#78] +(141) Exchange +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: hashpartitioning(ss_customer_sk#22, 5), true, [id=#66] -(148) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 +(142) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24] +Arguments: [ss_customer_sk#22 ASC NULLS FIRST], false, 0 -(149) Scan parquet default.customer -Output [1]: [c_customer_sk#28] +(143) Scan parquet default.customer +Output [1]: [c_customer_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(150) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#28] +(144) ColumnarToRow [codegen id : 4] +Input [1]: [c_customer_sk#26] -(151) Filter [codegen id : 4] -Input [1]: [c_customer_sk#28] -Condition : isnotnull(c_customer_sk#28) +(145) Filter [codegen id : 4] +Input [1]: [c_customer_sk#26] +Condition : isnotnull(c_customer_sk#26) -(152) Exchange -Input [1]: [c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), true, [id=#79] +(146) Exchange +Input [1]: [c_customer_sk#26] +Arguments: hashpartitioning(c_customer_sk#26, 5), true, [id=#67] -(153) Sort [codegen id : 5] -Input [1]: [c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +(147) Sort [codegen id : 5] +Input [1]: [c_customer_sk#26] +Arguments: [c_customer_sk#26 ASC NULLS FIRST], false, 0 -(154) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#24] -Right keys [1]: [c_customer_sk#28] +(148) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#22] +Right keys [1]: [c_customer_sk#26] Join condition: None -(155) Project [codegen id : 6] -Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#28] - -(156) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#28] -Keys [1]: [c_customer_sk#28] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#80, isEmpty#81] -Results [3]: [c_customer_sk#28, sum#82, isEmpty#83] - -(157) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#28, sum#82, isEmpty#83] -Keys [1]: [c_customer_sk#28] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#84] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#84 AS csales#85] - -(158) HashAggregate [codegen id : 6] -Input [1]: [csales#85] +(149) Project [codegen id : 6] +Output [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Input [4]: [ss_customer_sk#22, ss_quantity#23, ss_sales_price#24, c_customer_sk#26] + +(150) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#23, ss_sales_price#24, c_customer_sk#26] +Keys [1]: [c_customer_sk#26] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#68] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#23 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#24 as decimal(12,2)))), DecimalType(18,2), true))#68 AS csales#69] + +(151) HashAggregate [codegen id : 6] +Input [1]: [csales#69] Keys: [] -Functions [1]: [partial_max(csales#85)] -Aggregate Attributes [1]: [max#86] -Results [1]: [max#87] +Functions [1]: [partial_max(csales#69)] +Aggregate Attributes [1]: [max#70] +Results [1]: [max#71] -(159) Exchange -Input [1]: [max#87] -Arguments: SinglePartition, true, [id=#88] +(152) Exchange +Input [1]: [max#71] +Arguments: SinglePartition, true, [id=#72] -(160) HashAggregate [codegen id : 7] -Input [1]: [max#87] +(153) HashAggregate [codegen id : 7] +Input [1]: [max#71] Keys: [] -Functions [1]: [max(csales#85)] -Aggregate Attributes [1]: [max(csales#85)#89] -Results [1]: [max(csales#85)#89 AS tpcds_cmax#90] +Functions [1]: [max(csales#69)] +Aggregate Attributes [1]: [max(csales#69)#73] +Results [1]: [max(csales#69)#73 AS tpcds_cmax#74] -Subquery:2 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] +Subquery:2 Hosting operator id = 69 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] -Subquery:3 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] +Subquery:3 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] -Subquery:4 Hosting operator id = 126 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#37] +Subquery:4 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] 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 4279bf3e16a82..04aae428a4eef 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 @@ -35,109 +35,106 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Sort [item_sk] Project [item_sk] Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (4) - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_date_sk,d_date] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - WholeStageCodegen (15) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - Subquery #1 - WholeStageCodegen (7) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #9 - WholeStageCodegen (6) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] + HashAggregate [i_item_desc,i_item_sk,d_date] [count(1),item_sk,count(1),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] + WholeStageCodegen (5) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #10 - WholeStageCodegen (2) - Project [ss_customer_sk,ss_quantity,ss_sales_price] + Exchange [ss_item_sk] #4 + WholeStageCodegen (4) + Project [ss_item_sk,d_date] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Project [d_date_sk] + BroadcastExchange #5 + WholeStageCodegen (3) + Project [d_date_sk,d_date] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_year] InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] + WholeStageCodegen (7) + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #12 - WholeStageCodegen (4) - Filter [c_customer_sk] + Exchange [i_item_sk] #6 + WholeStageCodegen (6) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk] - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #7 - WholeStageCodegen (11) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] - InputAdapter - WholeStageCodegen (14) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (13) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk] + Scan parquet default.item [i_item_sk,i_item_desc] + WholeStageCodegen (15) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + Subquery #1 + WholeStageCodegen (7) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #9 + WholeStageCodegen (6) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #10 + WholeStageCodegen (2) + Project [ss_customer_sk,ss_quantity,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #12 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #7 + WholeStageCodegen (11) + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price] + InputAdapter + WholeStageCodegen (14) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (13) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk] InputAdapter BroadcastExchange #13 WholeStageCodegen (16) @@ -162,20 +159,19 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (21) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 - InputAdapter - WholeStageCodegen (23) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #8 + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (21) + Sort [ss_customer_sk] + InputAdapter + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 + InputAdapter + WholeStageCodegen (23) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk] #8 WholeStageCodegen (52) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter @@ -211,39 +207,37 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Sort [item_sk] Project [item_sk] Filter [count(1)] - HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] - HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] - Project [d_date,i_item_sk,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (31) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk,d_date] #4 - InputAdapter - WholeStageCodegen (33) - Sort [i_item_sk] - InputAdapter - ReusedExchange [i_item_sk,i_item_desc] #6 + HashAggregate [i_item_desc,i_item_sk,d_date] [count(1),item_sk,count(1),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (31) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,d_date] #4 + InputAdapter + WholeStageCodegen (33) + Sort [i_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_item_desc] #6 WholeStageCodegen (41) Sort [c_customer_sk] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (38) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 - InputAdapter - WholeStageCodegen (40) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #8 + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (38) + Sort [ss_customer_sk] + InputAdapter + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 + InputAdapter + WholeStageCodegen (40) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk] #8 InputAdapter ReusedExchange [d_date_sk] #13 InputAdapter @@ -257,17 +251,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (47) - Sort [ss_customer_sk] - InputAdapter - ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 - InputAdapter - WholeStageCodegen (49) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk] #8 + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (47) + Sort [ss_customer_sk] + InputAdapter + ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #7 + InputAdapter + WholeStageCodegen (49) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 7465ddae84e8a..20e5355eb4079 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -1,71 +1,69 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftSemi (57) - :- SortMergeJoin LeftSemi (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (65) ++- Exchange (64) + +- * HashAggregate (63) + +- * HashAggregate (62) + +- * HashAggregate (61) + +- * HashAggregate (60) + +- Exchange (59) + +- * HashAggregate (58) + +- SortMergeJoin LeftSemi (57) + :- SortMergeJoin LeftSemi (39) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * SortMergeJoin Inner (32) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- * Sort (31) + : +- ReusedExchange (30) + +- * Sort (56) + +- Exchange (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.web_sales (40) + : +- ReusedExchange (43) + +- * Sort (49) + +- ReusedExchange (48) (1) Scan parquet default.store_sales @@ -352,34 +350,20 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(63) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(64) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [1]: [count#35] -(66) Exchange +(64) Exchange Input [1]: [count#35] Arguments: SinglePartition, true, [id=#36] -(67) HashAggregate [codegen id : 26] +(65) HashAggregate [codegen id : 26] Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index 8dd59340cf069..13bb13c9dad65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -7,111 +7,109 @@ WholeStageCodegen (26) HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (24) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (24) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (7) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (15) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (13) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (9) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + WholeStageCodegen (23) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (22) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (21) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (17) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (20) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 74454cf32afd0..47e6fc236bfb7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,58 +1,56 @@ == Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftSemi BuildRight (44) - :- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- 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.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +* HashAggregate (52) ++- Exchange (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * BroadcastHashJoin LeftSemi BuildRight (44) + :- * BroadcastHashJoin LeftSemi BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- 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.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -287,34 +285,20 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(50) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(51) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(52) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#30] Results [1]: [count#31] -(53) Exchange +(51) Exchange Input [1]: [count#31] Arguments: SinglePartition, true, [id=#32] -(54) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 13] Input [1]: [count#31] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index a5b57a4ac9450..4faf16a139fca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -7,74 +7,72 @@ WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index d4c3d754860b1..2a2d6abdbc7a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * SortMergeJoin Inner (49) +TakeOrderedAndProject (50) ++- * Project (49) + +- * SortMergeJoin Inner (48) :- * Sort (14) : +- Exchange (13) : +- * Project (12) @@ -16,40 +16,39 @@ TakeOrderedAndProject (51) : +- * Filter (8) : +- * ColumnarToRow (7) : +- Scan parquet default.customer_address (6) - +- * Sort (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- * Project (44) - +- * SortMergeJoin Inner (43) - :- * Sort (40) - : +- Exchange (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (17) - : : : : +- * ColumnarToRow (16) - : : : : +- Scan parquet default.store_sales (15) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.date_dim (18) - : : +- BroadcastExchange (29) - : : +- * Project (28) - : : +- * Filter (27) - : : +- * ColumnarToRow (26) - : : +- Scan parquet default.store (25) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.household_demographics (32) - +- * Sort (42) - +- ReusedExchange (41) + +- * Sort (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * SortMergeJoin Inner (43) + :- * Sort (40) + : +- Exchange (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet default.store_sales (15) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet default.date_dim (18) + : : +- BroadcastExchange (29) + : : +- * Project (28) + : : +- * Filter (27) + : : +- * ColumnarToRow (26) + : : +- Scan parquet default.store (25) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.household_demographics (32) + +- * Sort (42) + +- ReusedExchange (41) (1) Scan parquet default.customer @@ -247,35 +246,28 @@ Input [7]: [ss_customer_sk#11, ss_addr_sk#13, ss_ticket_number#15, ss_coupon_amt (45) HashAggregate [codegen id : 14] Input [6]: [ss_customer_sk#11, ss_addr_sk#13, ss_ticket_number#15, ss_coupon_amt#16, ss_net_profit#17, ca_city#7] Keys [4]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#16)), partial_sum(UnscaledValue(ss_net_profit#17))] -Aggregate Attributes [2]: [sum#30, sum#31] -Results [6]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7, sum#32, sum#33] - -(46) HashAggregate [codegen id : 14] -Input [6]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7, sum#32, sum#33] -Keys [4]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#16)), sum(UnscaledValue(ss_net_profit#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#16))#34, sum(UnscaledValue(ss_net_profit#17))#35] -Results [5]: [ss_ticket_number#15, ss_customer_sk#11, ca_city#7 AS bought_city#36, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#16))#34,17,2) AS amt#37, MakeDecimal(sum(UnscaledValue(ss_net_profit#17))#35,17,2) AS profit#38] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#16))#30, sum(UnscaledValue(ss_net_profit#17))#31] +Results [5]: [ss_ticket_number#15, ss_customer_sk#11, ca_city#7 AS bought_city#32, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#16))#30,17,2) AS amt#33, MakeDecimal(sum(UnscaledValue(ss_net_profit#17))#31,17,2) AS profit#34] -(47) Exchange -Input [5]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#36, amt#37, profit#38] -Arguments: hashpartitioning(ss_customer_sk#11, 5), true, [id=#39] +(46) Exchange +Input [5]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#32, amt#33, profit#34] +Arguments: hashpartitioning(ss_customer_sk#11, 5), true, [id=#35] -(48) Sort [codegen id : 15] -Input [5]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#36, amt#37, profit#38] +(47) Sort [codegen id : 15] +Input [5]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#32, amt#33, profit#34] Arguments: [ss_customer_sk#11 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] +(48) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#11] -Join condition: NOT (ca_city#7 = bought_city#36) +Join condition: NOT (ca_city#7 = bought_city#32) -(50) Project [codegen id : 16] -Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#15, amt#37, profit#38] -Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#15, ss_customer_sk#11, bought_city#36, amt#37, profit#38] +(49) Project [codegen id : 16] +Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#32, ss_ticket_number#15, amt#33, profit#34] +Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#15, ss_customer_sk#11, bought_city#32, amt#33, profit#34] -(51) TakeOrderedAndProject -Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#15, amt#37, profit#38] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#36 ASC NULLS FIRST, ss_ticket_number#15 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#15, amt#37, profit#38] +(50) TakeOrderedAndProject +Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#32, ss_ticket_number#15, amt#33, profit#34] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#32 ASC NULLS FIRST, ss_ticket_number#15 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#32, ss_ticket_number#15, amt#33, profit#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt index 5eef8b789842c..6ec615fab4e21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt @@ -36,52 +36,51 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (14) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - SortMergeJoin [ss_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (11) - Sort [ss_addr_sk] - InputAdapter - Exchange [ss_addr_sk] #5 - WholeStageCodegen (10) - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Project [d_date_sk] - Filter [d_dow,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Project [s_store_sk] - Filter [s_city,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (9) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - WholeStageCodegen (13) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #3 + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + SortMergeJoin [ss_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_addr_sk] + InputAdapter + Exchange [ss_addr_sk] #5 + WholeStageCodegen (10) + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + Project [d_date_sk] + Filter [d_dow,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [s_store_sk] + Filter [s_city,s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_city] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (9) + Project [hd_demo_sk] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + WholeStageCodegen (13) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_city] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index d78565986bc0a..d7b2f0a14ed87 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -1,75 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (57) - : +- * BroadcastHashJoin Inner BuildRight (56) - : :- * Project (51) - : : +- * SortMergeJoin Inner (50) - : : :- * Sort (44) - : : : +- Exchange (43) - : : : +- * Project (42) - : : : +- * SortMergeJoin Inner (41) - : : : :- * Sort (35) - : : : : +- * HashAggregate (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * Project (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Project (20) - : : : : : +- * Filter (19) - : : : : : +- * ColumnarToRow (18) - : : : : : +- Scan parquet default.date_dim (17) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.customer (26) - : : : +- * Sort (40) - : : : +- Exchange (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.store_sales (36) - : : +- * Sort (49) - : : +- Exchange (48) - : : +- * Filter (47) - : : +- * ColumnarToRow (46) - : : +- Scan parquet default.customer_address (45) - : +- BroadcastExchange (55) - : +- * Filter (54) - : +- * ColumnarToRow (53) - : +- Scan parquet default.store (52) - +- BroadcastExchange (62) - +- * Project (61) - +- * Filter (60) - +- * ColumnarToRow (59) - +- Scan parquet default.date_dim (58) +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (50) + : : +- * SortMergeJoin Inner (49) + : : :- * Sort (43) + : : : +- Exchange (42) + : : : +- * Project (41) + : : : +- * SortMergeJoin Inner (40) + : : : :- * Sort (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * SortMergeJoin Inner (31) + : : : : :- * Sort (25) + : : : : : +- Exchange (24) + : : : : : +- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Project (16) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : : : :- Union (9) + : : : : : : : :- * Project (4) + : : : : : : : : +- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- * Project (8) + : : : : : : : +- * Filter (7) + : : : : : : : +- * ColumnarToRow (6) + : : : : : : : +- Scan parquet default.web_sales (5) + : : : : : : +- BroadcastExchange (14) + : : : : : : +- * Project (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (21) + : : : : : +- * Project (20) + : : : : : +- * Filter (19) + : : : : : +- * ColumnarToRow (18) + : : : : : +- Scan parquet default.date_dim (17) + : : : : +- * Sort (30) + : : : : +- Exchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.customer (26) + : : : +- * Sort (39) + : : : +- Exchange (38) + : : : +- * Filter (37) + : : : +- * ColumnarToRow (36) + : : : +- Scan parquet default.store_sales (35) + : : +- * Sort (48) + : : +- Exchange (47) + : : +- * Filter (46) + : : +- * ColumnarToRow (45) + : : +- Scan parquet default.customer_address (44) + : +- BroadcastExchange (54) + : +- * Filter (53) + : +- * ColumnarToRow (52) + : +- Scan parquet default.store (51) + +- BroadcastExchange (61) + +- * Project (60) + +- * Filter (59) + +- * ColumnarToRow (58) + +- Scan parquet default.date_dim (57) (1) Scan parquet default.catalog_sales @@ -218,273 +217,266 @@ Functions: [] Aggregate Attributes: [] Results [2]: [c_customer_sk#22, c_current_addr_sk#23] -(34) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Keys [2]: [c_customer_sk#22, c_current_addr_sk#23] -Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#22, c_current_addr_sk#23] - -(35) Sort [codegen id : 9] +(34) Sort [codegen id : 9] Input [2]: [c_customer_sk#22, c_current_addr_sk#23] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 -(36) Scan parquet default.store_sales +(35) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 10] Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -(38) Filter [codegen id : 10] +(37) Filter [codegen id : 10] Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] Condition : (isnotnull(ss_customer_sk#26) AND isnotnull(ss_sold_date_sk#25)) -(39) Exchange +(38) Exchange Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] Arguments: hashpartitioning(ss_customer_sk#26, 5), true, [id=#28] -(40) Sort [codegen id : 11] +(39) Sort [codegen id : 11] Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 12] +(40) SortMergeJoin [codegen id : 12] Left keys [1]: [c_customer_sk#22] Right keys [1]: [ss_customer_sk#26] Join condition: None -(42) Project [codegen id : 12] +(41) Project [codegen id : 12] Output [4]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27] Input [5]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -(43) Exchange +(42) Exchange Input [4]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27] Arguments: hashpartitioning(c_current_addr_sk#23, 5), true, [id=#29] -(44) Sort [codegen id : 13] +(43) Sort [codegen id : 13] Input [4]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27] Arguments: [c_current_addr_sk#23 ASC NULLS FIRST], false, 0 -(45) Scan parquet default.customer_address +(44) Scan parquet default.customer_address Output [3]: [ca_address_sk#30, ca_county#31, ca_state#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 14] +(45) ColumnarToRow [codegen id : 14] Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] -(47) Filter [codegen id : 14] +(46) Filter [codegen id : 14] Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] Condition : ((isnotnull(ca_address_sk#30) AND isnotnull(ca_county#31)) AND isnotnull(ca_state#32)) -(48) Exchange +(47) Exchange Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] Arguments: hashpartitioning(ca_address_sk#30, 5), true, [id=#33] -(49) Sort [codegen id : 15] +(48) Sort [codegen id : 15] Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] Arguments: [ca_address_sk#30 ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 18] +(49) SortMergeJoin [codegen id : 18] Left keys [1]: [c_current_addr_sk#23] Right keys [1]: [ca_address_sk#30] Join condition: None -(51) Project [codegen id : 18] +(50) Project [codegen id : 18] Output [5]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_county#31, ca_state#32] Input [7]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_address_sk#30, ca_county#31, ca_state#32] -(52) Scan parquet default.store +(51) Scan parquet default.store Output [2]: [s_county#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 16] +(52) ColumnarToRow [codegen id : 16] Input [2]: [s_county#34, s_state#35] -(54) Filter [codegen id : 16] +(53) Filter [codegen id : 16] Input [2]: [s_county#34, s_state#35] Condition : (isnotnull(s_county#34) AND isnotnull(s_state#35)) -(55) BroadcastExchange +(54) BroadcastExchange Input [2]: [s_county#34, s_state#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [id=#36] -(56) BroadcastHashJoin [codegen id : 18] +(55) BroadcastHashJoin [codegen id : 18] Left keys [2]: [ca_county#31, ca_state#32] Right keys [2]: [s_county#34, s_state#35] Join condition: None -(57) Project [codegen id : 18] +(56) Project [codegen id : 18] Output [3]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27] Input [7]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_county#31, ca_state#32, s_county#34, s_state#35] -(58) Scan parquet default.date_dim +(57) Scan parquet default.date_dim Output [2]: [d_date_sk#17, d_month_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 17] +(58) ColumnarToRow [codegen id : 17] Input [2]: [d_date_sk#17, d_month_seq#37] -(60) Filter [codegen id : 17] +(59) Filter [codegen id : 17] Input [2]: [d_date_sk#17, d_month_seq#37] Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#17)) -(61) Project [codegen id : 17] +(60) Project [codegen id : 17] Output [1]: [d_date_sk#17] Input [2]: [d_date_sk#17, d_month_seq#37] -(62) BroadcastExchange +(61) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] -(63) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 18] Left keys [1]: [ss_sold_date_sk#25] Right keys [1]: [d_date_sk#17] Join condition: None -(64) Project [codegen id : 18] +(63) Project [codegen id : 18] Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] Input [4]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, d_date_sk#17] -(65) HashAggregate [codegen id : 18] +(64) HashAggregate [codegen id : 18] Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] Keys [1]: [c_customer_sk#22] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] Aggregate Attributes [1]: [sum#43] Results [2]: [c_customer_sk#22, sum#44] -(66) Exchange +(65) Exchange Input [2]: [c_customer_sk#22, sum#44] Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#45] -(67) HashAggregate [codegen id : 19] +(66) HashAggregate [codegen id : 19] Input [2]: [c_customer_sk#22, sum#44] Keys [1]: [c_customer_sk#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#46] Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] -(68) HashAggregate [codegen id : 19] +(67) HashAggregate [codegen id : 19] Input [1]: [segment#47] Keys [1]: [segment#47] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#48] Results [2]: [segment#47, count#49] -(69) Exchange +(68) Exchange Input [2]: [segment#47, count#49] Arguments: hashpartitioning(segment#47, 5), true, [id=#50] -(70) HashAggregate [codegen id : 20] +(69) HashAggregate [codegen id : 20] Input [2]: [segment#47, count#49] Keys [1]: [segment#47] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#51] Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] -(71) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [3]: [segment#47, num_customers#52, segment_base#53] Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] ===== Subqueries ===== -Subquery:1 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (78) -+- Exchange (77) - +- * HashAggregate (76) - +- * Project (75) - +- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet default.date_dim (72) +Subquery:1 Hosting operator id = 59 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (77) ++- Exchange (76) + +- * HashAggregate (75) + +- * Project (74) + +- * Filter (73) + +- * ColumnarToRow (72) + +- Scan parquet default.date_dim (71) -(72) Scan parquet default.date_dim +(71) Scan parquet default.date_dim Output [3]: [d_month_seq#37, d_year#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 1] +(72) ColumnarToRow [codegen id : 1] Input [3]: [d_month_seq#37, d_year#18, d_moy#19] -(74) Filter [codegen id : 1] +(73) Filter [codegen id : 1] Input [3]: [d_month_seq#37, d_year#18, d_moy#19] Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) -(75) Project [codegen id : 1] +(74) Project [codegen id : 1] Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] Input [3]: [d_month_seq#37, d_year#18, d_moy#19] -(76) HashAggregate [codegen id : 1] +(75) HashAggregate [codegen id : 1] Input [1]: [(d_month_seq + 1)#54] Keys [1]: [(d_month_seq + 1)#54] Functions: [] Aggregate Attributes: [] Results [1]: [(d_month_seq + 1)#54] -(77) Exchange +(76) Exchange Input [1]: [(d_month_seq + 1)#54] Arguments: hashpartitioning((d_month_seq + 1)#54, 5), true, [id=#55] -(78) HashAggregate [codegen id : 2] +(77) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 1)#54] Keys [1]: [(d_month_seq + 1)#54] Functions: [] Aggregate Attributes: [] Results [1]: [(d_month_seq + 1)#54] -Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (85) -+- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * Filter (81) - +- * ColumnarToRow (80) - +- Scan parquet default.date_dim (79) +Subquery:2 Hosting operator id = 59 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (84) ++- Exchange (83) + +- * HashAggregate (82) + +- * Project (81) + +- * Filter (80) + +- * ColumnarToRow (79) + +- Scan parquet default.date_dim (78) -(79) Scan parquet default.date_dim +(78) Scan parquet default.date_dim Output [3]: [d_month_seq#37, d_year#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [3]: [d_month_seq#37, d_year#18, d_moy#19] -(81) Filter [codegen id : 1] +(80) Filter [codegen id : 1] Input [3]: [d_month_seq#37, d_year#18, d_moy#19] Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) -(82) Project [codegen id : 1] +(81) Project [codegen id : 1] Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] Input [3]: [d_month_seq#37, d_year#18, d_moy#19] -(83) HashAggregate [codegen id : 1] +(82) HashAggregate [codegen id : 1] Input [1]: [(d_month_seq + 3)#56] Keys [1]: [(d_month_seq + 3)#56] Functions: [] Aggregate Attributes: [] Results [1]: [(d_month_seq + 3)#56] -(84) Exchange +(83) Exchange Input [1]: [(d_month_seq + 3)#56] Arguments: hashpartitioning((d_month_seq + 3)#56, 5), true, [id=#57] -(85) HashAggregate [codegen id : 2] +(84) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 3)#56] Keys [1]: [(d_month_seq + 3)#56] Functions: [] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt index cb7130f53c9a9..0fe27e2eec40f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt @@ -28,59 +28,58 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (9) Sort [c_customer_sk] HashAggregate [c_customer_sk,c_current_addr_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - SortMergeJoin [customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - Sort [customer_sk] - InputAdapter - Exchange [customer_sk] #4 - WholeStageCodegen (5) - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - WholeStageCodegen (2) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [i_item_sk] - Filter [i_category,i_class,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - WholeStageCodegen (8) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #7 - WholeStageCodegen (7) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + SortMergeJoin [customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + Sort [customer_sk] + InputAdapter + Exchange [customer_sk] #4 + WholeStageCodegen (5) + Project [customer_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + InputAdapter + Union + WholeStageCodegen (1) + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + WholeStageCodegen (2) + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [i_item_sk] + Filter [i_category,i_class,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + WholeStageCodegen (8) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #7 + WholeStageCodegen (7) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index b3dc146e26e38..deb01185587d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * SortMergeJoin Inner (49) +TakeOrderedAndProject (50) ++- * Project (49) + +- * SortMergeJoin Inner (48) :- * Sort (14) : +- Exchange (13) : +- * Project (12) @@ -16,40 +16,39 @@ TakeOrderedAndProject (51) : +- * Filter (8) : +- * ColumnarToRow (7) : +- Scan parquet default.customer_address (6) - +- * Sort (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- * Project (44) - +- * SortMergeJoin Inner (43) - :- * Sort (40) - : +- Exchange (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (17) - : : : : +- * ColumnarToRow (16) - : : : : +- Scan parquet default.store_sales (15) - : : : +- BroadcastExchange (22) - : : : +- * Project (21) - : : : +- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet default.date_dim (18) - : : +- BroadcastExchange (29) - : : +- * Project (28) - : : +- * Filter (27) - : : +- * ColumnarToRow (26) - : : +- Scan parquet default.store (25) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.household_demographics (32) - +- * Sort (42) - +- ReusedExchange (41) + +- * Sort (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * SortMergeJoin Inner (43) + :- * Sort (40) + : +- Exchange (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (24) + : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : :- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet default.store_sales (15) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * Filter (20) + : : : +- * ColumnarToRow (19) + : : : +- Scan parquet default.date_dim (18) + : : +- BroadcastExchange (29) + : : +- * Project (28) + : : +- * Filter (27) + : : +- * ColumnarToRow (26) + : : +- Scan parquet default.store (25) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.household_demographics (32) + +- * Sort (42) + +- ReusedExchange (41) (1) Scan parquet default.customer @@ -247,35 +246,28 @@ Input [8]: [ss_customer_sk#11, ss_addr_sk#13, ss_ticket_number#15, ss_ext_sales_ (45) HashAggregate [codegen id : 14] Input [7]: [ss_customer_sk#11, ss_addr_sk#13, ss_ticket_number#15, ss_ext_sales_price#16, ss_ext_list_price#17, ss_ext_tax#18, ca_city#7] Keys [4]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#16)), partial_sum(UnscaledValue(ss_ext_list_price#17)), partial_sum(UnscaledValue(ss_ext_tax#18))] -Aggregate Attributes [3]: [sum#31, sum#32, sum#33] -Results [7]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7, sum#34, sum#35, sum#36] - -(46) HashAggregate [codegen id : 14] -Input [7]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7, sum#34, sum#35, sum#36] -Keys [4]: [ss_ticket_number#15, ss_customer_sk#11, ss_addr_sk#13, ca_city#7] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#16)), sum(UnscaledValue(ss_ext_list_price#17)), sum(UnscaledValue(ss_ext_tax#18))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#16))#37, sum(UnscaledValue(ss_ext_list_price#17))#38, sum(UnscaledValue(ss_ext_tax#18))#39] -Results [6]: [ss_ticket_number#15, ss_customer_sk#11, ca_city#7 AS bought_city#40, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#16))#37,17,2) AS extended_price#41, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#17))#38,17,2) AS list_price#42, MakeDecimal(sum(UnscaledValue(ss_ext_tax#18))#39,17,2) AS extended_tax#43] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#16))#31, sum(UnscaledValue(ss_ext_list_price#17))#32, sum(UnscaledValue(ss_ext_tax#18))#33] +Results [6]: [ss_ticket_number#15, ss_customer_sk#11, ca_city#7 AS bought_city#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#16))#31,17,2) AS extended_price#35, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#17))#32,17,2) AS list_price#36, MakeDecimal(sum(UnscaledValue(ss_ext_tax#18))#33,17,2) AS extended_tax#37] -(47) Exchange -Input [6]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#40, extended_price#41, list_price#42, extended_tax#43] -Arguments: hashpartitioning(ss_customer_sk#11, 5), true, [id=#44] +(46) Exchange +Input [6]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#34, extended_price#35, list_price#36, extended_tax#37] +Arguments: hashpartitioning(ss_customer_sk#11, 5), true, [id=#38] -(48) Sort [codegen id : 15] -Input [6]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#40, extended_price#41, list_price#42, extended_tax#43] +(47) Sort [codegen id : 15] +Input [6]: [ss_ticket_number#15, ss_customer_sk#11, bought_city#34, extended_price#35, list_price#36, extended_tax#37] Arguments: [ss_customer_sk#11 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] +(48) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#11] -Join condition: NOT (ca_city#7 = bought_city#40) +Join condition: NOT (ca_city#7 = bought_city#34) -(50) Project [codegen id : 16] -Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#15, extended_price#41, extended_tax#43, list_price#42] -Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#15, ss_customer_sk#11, bought_city#40, extended_price#41, list_price#42, extended_tax#43] +(49) Project [codegen id : 16] +Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#34, ss_ticket_number#15, extended_price#35, extended_tax#37, list_price#36] +Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#15, ss_customer_sk#11, bought_city#34, extended_price#35, list_price#36, extended_tax#37] -(51) TakeOrderedAndProject -Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#15, extended_price#41, extended_tax#43, list_price#42] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#15 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#15, extended_price#41, extended_tax#43, list_price#42] +(50) TakeOrderedAndProject +Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#34, ss_ticket_number#15, extended_price#35, extended_tax#37, list_price#36] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#15 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#34, ss_ticket_number#15, extended_price#35, extended_tax#37, list_price#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt index 64672d38be957..0ce4650de30ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt @@ -36,52 +36,51 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (14) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - SortMergeJoin [ss_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (11) - Sort [ss_addr_sk] - InputAdapter - Exchange [ss_addr_sk] #5 - WholeStageCodegen (10) - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - Project [d_date_sk] - Filter [d_dom,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - Project [s_store_sk] - Filter [s_city,s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (9) - Project [hd_demo_sk] - Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - WholeStageCodegen (13) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #3 + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + SortMergeJoin [ss_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_addr_sk] + InputAdapter + Exchange [ss_addr_sk] #5 + WholeStageCodegen (10) + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + Project [d_date_sk] + Filter [d_dom,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + Project [s_store_sk] + Filter [s_city,s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_city] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (9) + Project [hd_demo_sk] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + WholeStageCodegen (13) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_city] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 377bd36f520eb..2d67bc6b0fbc7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -1,71 +1,69 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftAnti (57) - :- SortMergeJoin LeftAnti (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (65) ++- Exchange (64) + +- * HashAggregate (63) + +- * HashAggregate (62) + +- * HashAggregate (61) + +- * HashAggregate (60) + +- Exchange (59) + +- * HashAggregate (58) + +- SortMergeJoin LeftAnti (57) + :- SortMergeJoin LeftAnti (39) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * SortMergeJoin Inner (32) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- * Sort (31) + : +- ReusedExchange (30) + +- * Sort (56) + +- Exchange (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.web_sales (40) + : +- ReusedExchange (43) + +- * Sort (49) + +- ReusedExchange (48) (1) Scan parquet default.store_sales @@ -352,34 +350,20 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(63) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(64) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [1]: [count#35] -(66) Exchange +(64) Exchange Input [1]: [count#35] Arguments: SinglePartition, true, [id=#36] -(67) HashAggregate [codegen id : 26] +(65) HashAggregate [codegen id : 26] Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt index 8dd59340cf069..13bb13c9dad65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt @@ -7,111 +7,109 @@ WholeStageCodegen (26) HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (24) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (24) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (7) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (15) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (13) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (9) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + WholeStageCodegen (23) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (22) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (21) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (17) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (20) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 3d59a670b7e8b..f6423420a3443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,58 +1,56 @@ == Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftAnti BuildRight (44) - :- * BroadcastHashJoin LeftAnti BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- 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.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +* HashAggregate (52) ++- Exchange (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * BroadcastHashJoin LeftAnti BuildRight (44) + :- * BroadcastHashJoin LeftAnti BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- 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.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -287,34 +285,20 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(50) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(51) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(52) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#30] Results [1]: [count#31] -(53) Exchange +(51) Exchange Input [1]: [count#31] Arguments: SinglePartition, true, [id=#32] -(54) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 13] Input [1]: [count#31] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index a5b57a4ac9450..4faf16a139fca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -7,74 +7,72 @@ WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 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 dad6098ce4685..22f2e58471089 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 @@ -1,114 +1,113 @@ == Physical Plan == -TakeOrderedAndProject (110) -+- * BroadcastHashJoin Inner BuildRight (109) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (109) ++- * BroadcastHashJoin Inner BuildRight (108) + :- * Project (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * Project (81) + : +- * BroadcastHashJoin Inner BuildRight (80) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- SortMergeJoin LeftSemi (63) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (62) + : : : +- Exchange (61) + : : : +- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - +- BroadcastExchange (108) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- SortMergeJoin LeftSemi (92) - : : :- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (91) - : : +- ReusedExchange (90) - : +- BroadcastExchange (97) - : +- * Project (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet default.date_dim (93) - +- ReusedExchange (100) + : : : +- BroadcastExchange (58) + : : : +- * HashAggregate (57) + : : : +- * HashAggregate (56) + : : : +- Exchange (55) + : : : +- * HashAggregate (54) + : : : +- SortMergeJoin LeftSemi (53) + : : : :- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- BroadcastExchange (68) + : : +- * Project (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.date_dim (64) + : +- BroadcastExchange (79) + : +- SortMergeJoin LeftSemi (78) + : :- * Sort (75) + : : +- Exchange (74) + : : +- * Filter (73) + : : +- * ColumnarToRow (72) + : : +- Scan parquet default.item (71) + : +- * Sort (77) + : +- ReusedExchange (76) + +- BroadcastExchange (107) + +- * Project (106) + +- * Filter (105) + +- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Project (101) + +- * BroadcastHashJoin Inner BuildRight (100) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- SortMergeJoin LeftSemi (91) + : : :- * Sort (88) + : : : +- ReusedExchange (87) + : : +- * Sort (90) + : : +- ReusedExchange (89) + : +- BroadcastExchange (96) + : +- * Project (95) + : +- * Filter (94) + : +- * ColumnarToRow (93) + : +- Scan parquet default.date_dim (92) + +- ReusedExchange (99) (1) Scan parquet default.store_sales @@ -369,441 +368,434 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(58) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(60) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(61) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(62) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(63) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#30, [id=#31])) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(67) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] -(70) BroadcastHashJoin [codegen id : 38] +(69) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(70) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(71) Scan parquet default.item 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_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(72) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(73) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(75) Exchange +(74) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#33] -(76) Sort [codegen id : 21] +(75) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(76) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(77) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(78) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(79) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] -(81) BroadcastHashJoin [codegen id : 38] +(80) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(81) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(82) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -(84) Exchange +(83) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#41] -(85) HashAggregate [codegen id : 78] +(84) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(86) Filter [codegen id : 78] +(85) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(87) Project [codegen id : 78] +(86) Project [codegen id : 78] Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(88) ReusedExchange [Reuses operator id: 4] +(87) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(89) Sort [codegen id : 40] +(88) Sort [codegen id : 40] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 62] +(89) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(91) Sort [codegen id : 56] +(90) Sort [codegen id : 56] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(91) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(93) Scan parquet default.date_dim +(92) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 57] +(93) ColumnarToRow [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] -(95) Filter [codegen id : 57] +(94) Filter [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 57] +(95) Project [codegen id : 57] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(97) BroadcastExchange +(96) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(98) BroadcastHashJoin [codegen id : 76] +(97) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(99) Project [codegen id : 76] +(98) Project [codegen id : 76] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(100) ReusedExchange [Reuses operator id: 80] +(99) ReusedExchange [Reuses operator id: 79] Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(101) BroadcastHashJoin [codegen id : 76] +(100) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#53] Join condition: None -(102) Project [codegen id : 76] +(101) Project [codegen id : 76] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(103) HashAggregate [codegen id : 76] +(102) HashAggregate [codegen id : 76] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -(104) Exchange +(103) Exchange Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), true, [id=#63] -(105) HashAggregate [codegen id : 77] +(104) HashAggregate [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(106) Filter [codegen id : 77] +(105) Filter [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(107) Project [codegen id : 77] +(106) Project [codegen id : 77] Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(108) BroadcastExchange +(107) BroadcastExchange Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] -(109) BroadcastHashJoin [codegen id : 78] +(108) BroadcastHashJoin [codegen id : 78] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Join condition: None -(110) TakeOrderedAndProject +(109) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (136) -+- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.store_sales (111) - : +- BroadcastExchange (118) - : +- * Project (117) - : +- * Filter (116) - : +- * ColumnarToRow (115) - : +- Scan parquet default.date_dim (114) - :- * Project (126) - : +- * BroadcastHashJoin Inner BuildRight (125) - : :- * Filter (123) - : : +- * ColumnarToRow (122) - : : +- Scan parquet default.catalog_sales (121) - : +- ReusedExchange (124) - +- * Project (132) - +- * BroadcastHashJoin Inner BuildRight (131) - :- * Filter (129) - : +- * ColumnarToRow (128) - : +- Scan parquet default.web_sales (127) - +- ReusedExchange (130) - - -(111) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 85 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (135) ++- Exchange (134) + +- * HashAggregate (133) + +- Union (132) + :- * Project (119) + : +- * BroadcastHashJoin Inner BuildRight (118) + : :- * Filter (112) + : : +- * ColumnarToRow (111) + : : +- Scan parquet default.store_sales (110) + : +- BroadcastExchange (117) + : +- * Project (116) + : +- * Filter (115) + : +- * ColumnarToRow (114) + : +- Scan parquet default.date_dim (113) + :- * Project (125) + : +- * BroadcastHashJoin Inner BuildRight (124) + : :- * Filter (122) + : : +- * ColumnarToRow (121) + : : +- Scan parquet default.catalog_sales (120) + : +- ReusedExchange (123) + +- * Project (131) + +- * BroadcastHashJoin Inner BuildRight (130) + :- * Filter (128) + : +- * ColumnarToRow (127) + : +- Scan parquet default.web_sales (126) + +- ReusedExchange (129) + + +(110) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(111) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(113) Filter [codegen id : 2] +(112) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(114) Scan parquet default.date_dim +(113) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(115) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(116) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(117) Project [codegen id : 1] +(116) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(118) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] -(119) BroadcastHashJoin [codegen id : 2] +(118) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(120) Project [codegen id : 2] +(119) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#72, ss_list_price#4 AS list_price#73] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(121) Scan parquet default.catalog_sales +(120) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] +(121) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] -(123) Filter [codegen id : 4] +(122) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Condition : isnotnull(cs_sold_date_sk#18) -(124) ReusedExchange [Reuses operator id: 118] +(123) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#10] -(125) BroadcastHashJoin [codegen id : 4] +(124) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(126) Project [codegen id : 4] +(125) Project [codegen id : 4] Output [2]: [cs_quantity#74 AS quantity#76, cs_list_price#75 AS list_price#77] Input [4]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75, d_date_sk#10] -(127) Scan parquet default.web_sales +(126) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 6] +(127) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] -(129) Filter [codegen id : 6] +(128) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Condition : isnotnull(ws_sold_date_sk#22) -(130) ReusedExchange [Reuses operator id: 118] +(129) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#10] -(131) BroadcastHashJoin [codegen id : 6] +(130) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(132) Project [codegen id : 6] +(131) Project [codegen id : 6] Output [2]: [ws_quantity#78 AS quantity#80, ws_list_price#79 AS list_price#81] Input [4]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79, d_date_sk#10] -(133) Union +(132) Union -(134) HashAggregate [codegen id : 7] +(133) HashAggregate [codegen id : 7] Input [2]: [quantity#72, list_price#73] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#82, count#83] Results [2]: [sum#84, count#85] -(135) Exchange +(134) Exchange Input [2]: [sum#84, count#85] Arguments: SinglePartition, true, [id=#86] -(136) HashAggregate [codegen id : 8] +(135) HashAggregate [codegen id : 8] Input [2]: [sum#84, count#85] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87 AS average_sales#88] -Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* Project (140) -+- * Filter (139) - +- * ColumnarToRow (138) - +- Scan parquet default.date_dim (137) +Subquery:2 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* Project (139) ++- * Filter (138) + +- * ColumnarToRow (137) + +- Scan parquet default.date_dim (136) -(137) Scan parquet default.date_dim +(136) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 1] +(137) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(139) Filter [codegen id : 1] +(138) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) -(140) Project [codegen id : 1] +(139) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] +Subquery:3 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:4 Hosting operator id = 94 Hosting Expression = Subquery scalar-subquery#50, [id=#51] +* Project (143) ++- * Filter (142) + +- * ColumnarToRow (141) + +- Scan parquet default.date_dim (140) -(141) Scan parquet default.date_dim +(140) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] +(141) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(143) Filter [codegen id : 1] +(142) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1998)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) -(144) Project [codegen id : 1] +(143) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] 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 d6b8ba4395d2e..db4f9c33ea415 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 @@ -81,81 +81,80 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_item_sk,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] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (13) + Exchange [i_brand_id,i_class_id,i_category_id] #9 + WholeStageCodegen (9) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #12 WholeStageCodegen (19) 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 1af2e69d57338..d4765c96e4926 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 @@ -1,104 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (99) ++- * BroadcastHashJoin Inner BuildRight (98) + :- * Project (76) + : +- * Filter (75) + : +- * HashAggregate (74) + : +- Exchange (73) + : +- * HashAggregate (72) + : +- * Project (71) + : +- * BroadcastHashJoin Inner BuildRight (70) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * BroadcastHashJoin LeftSemi BuildRight (56) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (55) + : : : +- * Project (54) + : : : +- * BroadcastHashJoin Inner BuildRight (53) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) + : : : +- BroadcastExchange (52) + : : : +- * HashAggregate (51) + : : : +- * HashAggregate (50) + : : : +- Exchange (49) + : : : +- * HashAggregate (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (39) + : : : : : +- * ColumnarToRow (38) + : : : : : +- Scan parquet default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (62) + : : +- * BroadcastHashJoin LeftSemi BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.item (57) + : : +- ReusedExchange (60) + : +- BroadcastExchange (69) + : +- * Project (68) + : +- * Filter (67) + : +- * ColumnarToRow (66) + : +- Scan parquet default.date_dim (65) + +- BroadcastExchange (97) + +- * Project (96) + +- * Filter (95) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * Project (91) + +- * BroadcastHashJoin Inner BuildRight (90) + :- * Project (84) + : +- * BroadcastHashJoin Inner BuildRight (83) + : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : :- * Filter (79) + : : : +- * ColumnarToRow (78) + : : : +- Scan parquet default.store_sales (77) + : : +- ReusedExchange (80) + : +- ReusedExchange (82) + +- BroadcastExchange (89) + +- * Project (88) + +- * Filter (87) + +- * ColumnarToRow (86) + +- Scan parquet default.date_dim (85) (1) Scan parquet default.store_sales @@ -335,428 +334,421 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(52) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(53) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(54) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(55) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(57) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(58) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(59) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) -(61) ReusedExchange [Reuses operator id: 56] +(60) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(61) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(62) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(65) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(66) ColumnarToRow [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] -(68) Filter [codegen id : 24] +(67) Filter [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#29, [id=#30])) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(68) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(70) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] -(71) BroadcastHashJoin [codegen id : 25] +(70) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(71) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(72) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] -(74) Exchange +(73) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#38] -(75) HashAggregate [codegen id : 52] +(74) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(76) Filter [codegen id : 52] +(75) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(77) Project [codegen id : 52] +(76) Project [codegen id : 52] Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(78) Scan parquet default.store_sales +(77) Scan parquet default.store_sales Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 50] +(78) ColumnarToRow [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(80) Filter [codegen id : 50] +(79) Filter [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(81) ReusedExchange [Reuses operator id: 56] +(80) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 50] +(81) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(82) ReusedExchange [Reuses operator id: 62] Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(84) BroadcastHashJoin [codegen id : 50] +(83) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#47] Join condition: None -(85) Project [codegen id : 50] +(84) Project [codegen id : 50] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(86) Scan parquet default.date_dim +(85) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 49] +(86) ColumnarToRow [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] -(88) Filter [codegen id : 49] +(87) Filter [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#10)) -(89) Project [codegen id : 49] +(88) Project [codegen id : 49] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(90) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(91) BroadcastHashJoin [codegen id : 50] +(90) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(92) Project [codegen id : 50] +(91) Project [codegen id : 50] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50, d_date_sk#10] -(93) HashAggregate [codegen id : 50] +(92) HashAggregate [codegen id : 50] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] -(94) Exchange +(93) Exchange Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), true, [id=#60] -(95) HashAggregate [codegen id : 51] +(94) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(96) Filter [codegen id : 51] +(95) Filter [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(97) Project [codegen id : 51] +(96) Project [codegen id : 51] Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(98) BroadcastExchange +(97) BroadcastExchange Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] -(99) BroadcastHashJoin [codegen id : 52] +(98) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Join condition: None -(100) TakeOrderedAndProject +(99) TakeOrderedAndProject Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * 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) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.catalog_sales (111) - : +- ReusedExchange (114) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * Filter (119) - : +- * ColumnarToRow (118) - : +- Scan parquet default.web_sales (117) - +- ReusedExchange (120) - - -(101) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 75 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* HashAggregate (125) ++- Exchange (124) + +- * HashAggregate (123) + +- Union (122) + :- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * Filter (102) + : : +- * ColumnarToRow (101) + : : +- Scan parquet default.store_sales (100) + : +- BroadcastExchange (107) + : +- * Project (106) + : +- * Filter (105) + : +- * ColumnarToRow (104) + : +- Scan parquet default.date_dim (103) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- * Filter (112) + : : +- * ColumnarToRow (111) + : : +- Scan parquet default.catalog_sales (110) + : +- ReusedExchange (113) + +- * Project (121) + +- * BroadcastHashJoin Inner BuildRight (120) + :- * Filter (118) + : +- * ColumnarToRow (117) + : +- Scan parquet default.web_sales (116) + +- ReusedExchange (119) + + +(100) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] +(101) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(103) Filter [codegen id : 2] +(102) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(104) Scan parquet default.date_dim +(103) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(104) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(106) Filter [codegen id : 1] +(105) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(107) Project [codegen id : 1] +(106) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(108) BroadcastExchange +(107) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] -(109) BroadcastHashJoin [codegen id : 2] +(108) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(110) Project [codegen id : 2] +(109) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#69, ss_list_price#4 AS list_price#70] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(111) Scan parquet default.catalog_sales +(110) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] +(111) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] -(113) Filter [codegen id : 4] +(112) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Condition : isnotnull(cs_sold_date_sk#16) -(114) ReusedExchange [Reuses operator id: 108] +(113) ReusedExchange [Reuses operator id: 107] Output [1]: [d_date_sk#10] -(115) BroadcastHashJoin [codegen id : 4] +(114) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(116) Project [codegen id : 4] +(115) Project [codegen id : 4] Output [2]: [cs_quantity#71 AS quantity#73, cs_list_price#72 AS list_price#74] Input [4]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72, d_date_sk#10] -(117) Scan parquet default.web_sales +(116) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 6] +(117) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] -(119) Filter [codegen id : 6] +(118) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Condition : isnotnull(ws_sold_date_sk#20) -(120) ReusedExchange [Reuses operator id: 108] +(119) ReusedExchange [Reuses operator id: 107] Output [1]: [d_date_sk#10] -(121) BroadcastHashJoin [codegen id : 6] +(120) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(122) Project [codegen id : 6] +(121) Project [codegen id : 6] Output [2]: [ws_quantity#75 AS quantity#77, ws_list_price#76 AS list_price#78] Input [4]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76, d_date_sk#10] -(123) Union +(122) Union -(124) HashAggregate [codegen id : 7] +(123) HashAggregate [codegen id : 7] Input [2]: [quantity#69, list_price#70] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#79, count#80] Results [2]: [sum#81, count#82] -(125) Exchange +(124) Exchange Input [2]: [sum#81, count#82] Arguments: SinglePartition, true, [id=#83] -(126) HashAggregate [codegen id : 8] +(125) HashAggregate [codegen id : 8] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS average_sales#85] -Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* Project (129) ++- * Filter (128) + +- * ColumnarToRow (127) + +- Scan parquet default.date_dim (126) -(127) Scan parquet default.date_dim +(126) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 1] +(127) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(129) Filter [codegen id : 1] +(128) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) -(130) Project [codegen id : 1] +(129) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:3 Hosting operator id = 95 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (134) -+- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) +Subquery:4 Hosting operator id = 87 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* Project (133) ++- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet default.date_dim (130) -(131) Scan parquet default.date_dim +(130) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 1] +(131) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(133) Filter [codegen id : 1] +(132) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1998)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) -(134) Project [codegen id : 1] +(133) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] 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 7bbf83e3de707..9bb0d49edff50 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 @@ -73,71 +73,70 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #4 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #4 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_item_sk,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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) + BroadcastExchange #7 + WholeStageCodegen (5) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + BroadcastExchange #8 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (23) 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 38292528b42fc..60f79988d7cd8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -1,226 +1,225 @@ == Physical Plan == -TakeOrderedAndProject (222) -+- * HashAggregate (221) - +- Exchange (220) - +- * HashAggregate (219) - +- Union (218) - :- * HashAggregate (198) - : +- Exchange (197) - : +- * HashAggregate (196) - : +- Union (195) - : :- * HashAggregate (175) - : : +- Exchange (174) - : : +- * HashAggregate (173) - : : +- Union (172) - : : :- * HashAggregate (152) - : : : +- Exchange (151) - : : : +- * HashAggregate (150) - : : : +- Union (149) - : : : :- * HashAggregate (129) - : : : : +- Exchange (128) - : : : : +- * HashAggregate (127) - : : : : +- Union (126) - : : : : :- * Project (87) - : : : : : +- * Filter (86) - : : : : : +- * HashAggregate (85) - : : : : : +- Exchange (84) - : : : : : +- * HashAggregate (83) - : : : : : +- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (71) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (70) - : : : : : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (221) ++- * HashAggregate (220) + +- Exchange (219) + +- * HashAggregate (218) + +- Union (217) + :- * HashAggregate (197) + : +- Exchange (196) + : +- * HashAggregate (195) + : +- Union (194) + : :- * HashAggregate (174) + : : +- Exchange (173) + : : +- * HashAggregate (172) + : : +- Union (171) + : : :- * HashAggregate (151) + : : : +- Exchange (150) + : : : +- * HashAggregate (149) + : : : +- Union (148) + : : : :- * HashAggregate (128) + : : : : +- Exchange (127) + : : : : +- * HashAggregate (126) + : : : : +- Union (125) + : : : : :- * Project (86) + : : : : : +- * Filter (85) + : : : : : +- * HashAggregate (84) + : : : : : +- Exchange (83) + : : : : : +- * HashAggregate (82) + : : : : : +- * Project (81) + : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : :- * Project (70) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) + : : : : : : :- SortMergeJoin LeftSemi (63) : : : : : : : :- * Sort (5) : : : : : : : : +- Exchange (4) : : : : : : : : +- * Filter (3) : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : : +- * Sort (63) - : : : : : : : +- Exchange (62) - : : : : : : : +- * Project (61) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : : : : +- * Sort (62) + : : : : : : : +- Exchange (61) + : : : : : : : +- * Project (60) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (59) : : : : : : : :- * Filter (8) : : : : : : : : +- * ColumnarToRow (7) : : : : : : : : +- Scan parquet default.item (6) - : : : : : : : +- BroadcastExchange (59) - : : : : : : : +- * HashAggregate (58) - : : : : : : : +- * HashAggregate (57) - : : : : : : : +- * HashAggregate (56) - : : : : : : : +- Exchange (55) - : : : : : : : +- * HashAggregate (54) - : : : : : : : +- SortMergeJoin LeftSemi (53) - : : : : : : : :- SortMergeJoin LeftSemi (41) - : : : : : : : : :- * Sort (26) - : : : : : : : : : +- Exchange (25) - : : : : : : : : : +- * Project (24) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : : : : :- * Project (18) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : : : : :- * Filter (11) - : : : : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : : : : +- BroadcastExchange (16) - : : : : : : : : : : +- * Project (15) - : : : : : : : : : : +- * Filter (14) - : : : : : : : : : : +- * ColumnarToRow (13) - : : : : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : : : : +- BroadcastExchange (22) - : : : : : : : : : +- * Filter (21) - : : : : : : : : : +- * ColumnarToRow (20) - : : : : : : : : : +- Scan parquet default.item (19) - : : : : : : : : +- * Sort (40) - : : : : : : : : +- Exchange (39) - : : : : : : : : +- * Project (38) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : : : : :- * Project (32) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : : : : :- * Filter (29) - : : : : : : : : : : +- * ColumnarToRow (28) - : : : : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : : : : +- ReusedExchange (30) - : : : : : : : : +- BroadcastExchange (36) - : : : : : : : : +- * Filter (35) - : : : : : : : : +- * ColumnarToRow (34) - : : : : : : : : +- Scan parquet default.item (33) - : : : : : : : +- * Sort (52) - : : : : : : : +- Exchange (51) - : : : : : : : +- * Project (50) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : : :- * Project (47) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : : : :- * Filter (44) - : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : +- Scan parquet default.web_sales (42) - : : : : : : : : +- ReusedExchange (45) - : : : : : : : +- ReusedExchange (48) - : : : : : : +- BroadcastExchange (69) - : : : : : : +- * Project (68) - : : : : : : +- * Filter (67) - : : : : : : +- * ColumnarToRow (66) - : : : : : : +- Scan parquet default.date_dim (65) - : : : : : +- BroadcastExchange (80) - : : : : : +- SortMergeJoin LeftSemi (79) - : : : : : :- * Sort (76) - : : : : : : +- Exchange (75) - : : : : : : +- * Filter (74) - : : : : : : +- * ColumnarToRow (73) - : : : : : : +- Scan parquet default.item (72) - : : : : : +- * Sort (78) - : : : : : +- ReusedExchange (77) - : : : : :- * Project (106) - : : : : : +- * Filter (105) - : : : : : +- * HashAggregate (104) - : : : : : +- Exchange (103) - : : : : : +- * HashAggregate (102) - : : : : : +- * Project (101) - : : : : : +- * BroadcastHashJoin Inner BuildRight (100) - : : : : : :- * Project (98) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : : : : :- SortMergeJoin LeftSemi (95) - : : : : : : : :- * Sort (92) - : : : : : : : : +- Exchange (91) - : : : : : : : : +- * Filter (90) - : : : : : : : : +- * ColumnarToRow (89) - : : : : : : : : +- Scan parquet default.catalog_sales (88) - : : : : : : : +- * Sort (94) - : : : : : : : +- ReusedExchange (93) - : : : : : : +- ReusedExchange (96) - : : : : : +- ReusedExchange (99) - : : : : +- * Project (125) - : : : : +- * Filter (124) - : : : : +- * HashAggregate (123) - : : : : +- Exchange (122) - : : : : +- * HashAggregate (121) - : : : : +- * Project (120) - : : : : +- * BroadcastHashJoin Inner BuildRight (119) - : : : : :- * Project (117) - : : : : : +- * BroadcastHashJoin Inner BuildRight (116) - : : : : : :- SortMergeJoin LeftSemi (114) - : : : : : : :- * Sort (111) - : : : : : : : +- Exchange (110) - : : : : : : : +- * Filter (109) - : : : : : : : +- * ColumnarToRow (108) - : : : : : : : +- Scan parquet default.web_sales (107) - : : : : : : +- * Sort (113) - : : : : : : +- ReusedExchange (112) - : : : : : +- ReusedExchange (115) - : : : : +- ReusedExchange (118) - : : : +- * HashAggregate (148) - : : : +- Exchange (147) - : : : +- * HashAggregate (146) - : : : +- * HashAggregate (145) - : : : +- Exchange (144) - : : : +- * HashAggregate (143) - : : : +- Union (142) - : : : :- * Project (133) - : : : : +- * Filter (132) - : : : : +- * HashAggregate (131) - : : : : +- ReusedExchange (130) - : : : :- * Project (137) - : : : : +- * Filter (136) - : : : : +- * HashAggregate (135) - : : : : +- ReusedExchange (134) - : : : +- * Project (141) - : : : +- * Filter (140) - : : : +- * HashAggregate (139) - : : : +- ReusedExchange (138) - : : +- * HashAggregate (171) - : : +- Exchange (170) - : : +- * HashAggregate (169) - : : +- * HashAggregate (168) - : : +- Exchange (167) - : : +- * HashAggregate (166) - : : +- Union (165) - : : :- * Project (156) - : : : +- * Filter (155) - : : : +- * HashAggregate (154) - : : : +- ReusedExchange (153) - : : :- * Project (160) - : : : +- * Filter (159) - : : : +- * HashAggregate (158) - : : : +- ReusedExchange (157) - : : +- * Project (164) - : : +- * Filter (163) - : : +- * HashAggregate (162) - : : +- ReusedExchange (161) - : +- * HashAggregate (194) - : +- Exchange (193) - : +- * HashAggregate (192) - : +- * HashAggregate (191) - : +- Exchange (190) - : +- * HashAggregate (189) - : +- Union (188) - : :- * Project (179) - : : +- * Filter (178) - : : +- * HashAggregate (177) - : : +- ReusedExchange (176) - : :- * Project (183) - : : +- * Filter (182) - : : +- * HashAggregate (181) - : : +- ReusedExchange (180) - : +- * Project (187) - : +- * Filter (186) - : +- * HashAggregate (185) - : +- ReusedExchange (184) - +- * HashAggregate (217) - +- Exchange (216) - +- * HashAggregate (215) - +- * HashAggregate (214) - +- Exchange (213) - +- * HashAggregate (212) - +- Union (211) - :- * Project (202) - : +- * Filter (201) - : +- * HashAggregate (200) - : +- ReusedExchange (199) - :- * Project (206) - : +- * Filter (205) - : +- * HashAggregate (204) - : +- ReusedExchange (203) - +- * Project (210) - +- * Filter (209) - +- * HashAggregate (208) - +- ReusedExchange (207) + : : : : : : : +- BroadcastExchange (58) + : : : : : : : +- * HashAggregate (57) + : : : : : : : +- * HashAggregate (56) + : : : : : : : +- Exchange (55) + : : : : : : : +- * HashAggregate (54) + : : : : : : : +- SortMergeJoin LeftSemi (53) + : : : : : : : :- SortMergeJoin LeftSemi (41) + : : : : : : : : :- * Sort (26) + : : : : : : : : : +- Exchange (25) + : : : : : : : : : +- * Project (24) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : : : : :- * Project (18) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : : : : : :- * Filter (11) + : : : : : : : : : : : +- * ColumnarToRow (10) + : : : : : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : : : : : +- BroadcastExchange (16) + : : : : : : : : : : +- * Project (15) + : : : : : : : : : : +- * Filter (14) + : : : : : : : : : : +- * ColumnarToRow (13) + : : : : : : : : : : +- Scan parquet default.date_dim (12) + : : : : : : : : : +- BroadcastExchange (22) + : : : : : : : : : +- * Filter (21) + : : : : : : : : : +- * ColumnarToRow (20) + : : : : : : : : : +- Scan parquet default.item (19) + : : : : : : : : +- * Sort (40) + : : : : : : : : +- Exchange (39) + : : : : : : : : +- * Project (38) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : : : : : :- * Project (32) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : : : : :- * Filter (29) + : : : : : : : : : : +- * ColumnarToRow (28) + : : : : : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : : : : : +- ReusedExchange (30) + : : : : : : : : +- BroadcastExchange (36) + : : : : : : : : +- * Filter (35) + : : : : : : : : +- * ColumnarToRow (34) + : : : : : : : : +- Scan parquet default.item (33) + : : : : : : : +- * Sort (52) + : : : : : : : +- Exchange (51) + : : : : : : : +- * Project (50) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : : : : :- * Project (47) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : : : : : :- * Filter (44) + : : : : : : : : : +- * ColumnarToRow (43) + : : : : : : : : : +- Scan parquet default.web_sales (42) + : : : : : : : : +- ReusedExchange (45) + : : : : : : : +- ReusedExchange (48) + : : : : : : +- BroadcastExchange (68) + : : : : : : +- * Project (67) + : : : : : : +- * Filter (66) + : : : : : : +- * ColumnarToRow (65) + : : : : : : +- Scan parquet default.date_dim (64) + : : : : : +- BroadcastExchange (79) + : : : : : +- SortMergeJoin LeftSemi (78) + : : : : : :- * Sort (75) + : : : : : : +- Exchange (74) + : : : : : : +- * Filter (73) + : : : : : : +- * ColumnarToRow (72) + : : : : : : +- Scan parquet default.item (71) + : : : : : +- * Sort (77) + : : : : : +- ReusedExchange (76) + : : : : :- * Project (105) + : : : : : +- * Filter (104) + : : : : : +- * HashAggregate (103) + : : : : : +- Exchange (102) + : : : : : +- * HashAggregate (101) + : : : : : +- * Project (100) + : : : : : +- * BroadcastHashJoin Inner BuildRight (99) + : : : : : :- * Project (97) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (96) + : : : : : : :- SortMergeJoin LeftSemi (94) + : : : : : : : :- * Sort (91) + : : : : : : : : +- Exchange (90) + : : : : : : : : +- * Filter (89) + : : : : : : : : +- * ColumnarToRow (88) + : : : : : : : : +- Scan parquet default.catalog_sales (87) + : : : : : : : +- * Sort (93) + : : : : : : : +- ReusedExchange (92) + : : : : : : +- ReusedExchange (95) + : : : : : +- ReusedExchange (98) + : : : : +- * Project (124) + : : : : +- * Filter (123) + : : : : +- * HashAggregate (122) + : : : : +- Exchange (121) + : : : : +- * HashAggregate (120) + : : : : +- * Project (119) + : : : : +- * BroadcastHashJoin Inner BuildRight (118) + : : : : :- * Project (116) + : : : : : +- * BroadcastHashJoin Inner BuildRight (115) + : : : : : :- SortMergeJoin LeftSemi (113) + : : : : : : :- * Sort (110) + : : : : : : : +- Exchange (109) + : : : : : : : +- * Filter (108) + : : : : : : : +- * ColumnarToRow (107) + : : : : : : : +- Scan parquet default.web_sales (106) + : : : : : : +- * Sort (112) + : : : : : : +- ReusedExchange (111) + : : : : : +- ReusedExchange (114) + : : : : +- ReusedExchange (117) + : : : +- * HashAggregate (147) + : : : +- Exchange (146) + : : : +- * HashAggregate (145) + : : : +- * HashAggregate (144) + : : : +- Exchange (143) + : : : +- * HashAggregate (142) + : : : +- Union (141) + : : : :- * Project (132) + : : : : +- * Filter (131) + : : : : +- * HashAggregate (130) + : : : : +- ReusedExchange (129) + : : : :- * Project (136) + : : : : +- * Filter (135) + : : : : +- * HashAggregate (134) + : : : : +- ReusedExchange (133) + : : : +- * Project (140) + : : : +- * Filter (139) + : : : +- * HashAggregate (138) + : : : +- ReusedExchange (137) + : : +- * HashAggregate (170) + : : +- Exchange (169) + : : +- * HashAggregate (168) + : : +- * HashAggregate (167) + : : +- Exchange (166) + : : +- * HashAggregate (165) + : : +- Union (164) + : : :- * Project (155) + : : : +- * Filter (154) + : : : +- * HashAggregate (153) + : : : +- ReusedExchange (152) + : : :- * Project (159) + : : : +- * Filter (158) + : : : +- * HashAggregate (157) + : : : +- ReusedExchange (156) + : : +- * Project (163) + : : +- * Filter (162) + : : +- * HashAggregate (161) + : : +- ReusedExchange (160) + : +- * HashAggregate (193) + : +- Exchange (192) + : +- * HashAggregate (191) + : +- * HashAggregate (190) + : +- Exchange (189) + : +- * HashAggregate (188) + : +- Union (187) + : :- * Project (178) + : : +- * Filter (177) + : : +- * HashAggregate (176) + : : +- ReusedExchange (175) + : :- * Project (182) + : : +- * Filter (181) + : : +- * HashAggregate (180) + : : +- ReusedExchange (179) + : +- * Project (186) + : +- * Filter (185) + : +- * HashAggregate (184) + : +- ReusedExchange (183) + +- * HashAggregate (216) + +- Exchange (215) + +- * HashAggregate (214) + +- * HashAggregate (213) + +- Exchange (212) + +- * HashAggregate (211) + +- Union (210) + :- * Project (201) + : +- * Filter (200) + : +- * HashAggregate (199) + : +- ReusedExchange (198) + :- * Project (205) + : +- * Filter (204) + : +- * HashAggregate (203) + : +- ReusedExchange (202) + +- * Project (209) + +- * Filter (208) + +- * HashAggregate (207) + +- ReusedExchange (206) (1) Scan parquet default.store_sales @@ -481,980 +480,973 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(58) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(60) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(61) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(62) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(63) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#29] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(65) ColumnarToRow [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(67) Filter [codegen id : 19] +(66) Filter [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2000)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(67) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(70) BroadcastHashJoin [codegen id : 38] +(69) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(70) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(71) Scan parquet default.item 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_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(72) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(73) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(74) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#31] -(76) Sort [codegen id : 21] +(75) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(76) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(77) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(78) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(79) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(81) BroadcastHashJoin [codegen id : 38] +(80) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(81) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(82) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -(84) Exchange +(83) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#39] -(85) HashAggregate [codegen id : 39] +(84) HashAggregate [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(86) Filter [codegen id : 39] +(85) Filter [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(87) Project [codegen id : 39] +(86) Project [codegen id : 39] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(88) Scan parquet default.catalog_sales +(87) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] +(88) ColumnarToRow [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -(90) Filter [codegen id : 40] +(89) Filter [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange +(90) Exchange Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: hashpartitioning(cs_item_sk#19, 5), true, [id=#50] -(92) Sort [codegen id : 41] +(91) Sort [codegen id : 41] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] +(92) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(94) Sort [codegen id : 57] +(93) Sort [codegen id : 57] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(94) SortMergeJoin Left keys [1]: [cs_item_sk#19] Right keys [1]: [ss_item_sk#27] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(95) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] +(97) Project [codegen id : 77] Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(98) ReusedExchange [Reuses operator id: 79] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(99) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] +(100) Project [codegen id : 77] Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] +(101) HashAggregate [codegen id : 77] Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -(103) Exchange +(102) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#57] -(104) HashAggregate [codegen id : 78] +(103) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(105) Filter [codegen id : 78] +(104) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(106) Project [codegen id : 78] +(105) Project [codegen id : 78] Output [6]: [catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(107) Scan parquet default.web_sales +(106) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] +(107) ColumnarToRow [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -(109) Filter [codegen id : 79] +(108) Filter [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) -(110) Exchange +(109) Exchange Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: hashpartitioning(ws_item_sk#23, 5), true, [id=#66] -(111) Sort [codegen id : 80] +(110) Sort [codegen id : 80] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] +(111) ReusedExchange [Reuses operator id: 61] Output [1]: [ss_item_sk#27] -(113) Sort [codegen id : 96] +(112) Sort [codegen id : 96] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin +(113) SortMergeJoin Left keys [1]: [ws_item_sk#23] Right keys [1]: [ss_item_sk#27] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(114) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] +(115) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] +(116) Project [codegen id : 116] Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(117) ReusedExchange [Reuses operator id: 79] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] +(118) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_item_sk#23] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] +(119) Project [codegen id : 116] Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] +(120) HashAggregate [codegen id : 116] Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -(122) Exchange +(121) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#73] -(123) HashAggregate [codegen id : 117] +(122) HashAggregate [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(124) Filter [codegen id : 117] +(123) Filter [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(125) Project [codegen id : 117] +(124) Project [codegen id : 117] Output [6]: [web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(126) Union +(125) Union -(127) HashAggregate [codegen id : 118] +(126) HashAggregate [codegen id : 118] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -(128) Exchange +(127) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#86] -(129) HashAggregate [codegen id : 119] +(128) HashAggregate [codegen id : 119] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#87, sum(number_sales#43)#88] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#42)#87 AS sum_sales#89, sum(number_sales#43)#88 AS number_sales#90] -(130) ReusedExchange [Reuses operator id: 84] +(129) ReusedExchange [Reuses operator id: 83] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] -(131) HashAggregate [codegen id : 158] +(130) HashAggregate [codegen id : 158] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94, count(1)#95] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#42, count(1)#95 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -(132) Filter [codegen id : 158] +(131) Filter [codegen id : 158] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(133) Project [codegen id : 158] +(132) Project [codegen id : 158] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -(134) ReusedExchange [Reuses operator id: 103] +(133) ReusedExchange [Reuses operator id: 102] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] -(135) HashAggregate [codegen id : 197] +(134) HashAggregate [codegen id : 197] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100, count(1)#101] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#60, count(1)#101 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -(136) Filter [codegen id : 197] +(135) Filter [codegen id : 197] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(137) Project [codegen id : 197] +(136) Project [codegen id : 197] Output [6]: [catalog AS channel#103, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -(138) ReusedExchange [Reuses operator id: 122] +(137) ReusedExchange [Reuses operator id: 121] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] -(139) HashAggregate [codegen id : 236] +(138) HashAggregate [codegen id : 236] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#76, count(1)#108 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -(140) Filter [codegen id : 236] +(139) Filter [codegen id : 236] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(141) Project [codegen id : 236] +(140) Project [codegen id : 236] Output [6]: [web AS channel#110, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -(142) Union +(141) Union -(143) HashAggregate [codegen id : 237] +(142) HashAggregate [codegen id : 237] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] -(144) Exchange +(143) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#117] -(145) HashAggregate [codegen id : 238] +(144) HashAggregate [codegen id : 238] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#118, sum(number_sales#43)#119] Results [5]: [channel#47, i_brand_id#7, i_class_id#8, sum(sales#42)#118 AS sum_sales#89, sum(number_sales#43)#119 AS number_sales#90] -(146) HashAggregate [codegen id : 238] +(145) HashAggregate [codegen id : 238] Input [5]: [channel#47, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90] Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] -(147) Exchange +(146) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, 5), true, [id=#126] -(148) HashAggregate [codegen id : 239] +(147) HashAggregate [codegen id : 239] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#127, sum(number_sales#90)#128] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, null AS i_category_id#129, sum(sum_sales#89)#127 AS sum(sum_sales)#130, sum(number_sales#90)#128 AS sum(number_sales)#131] -(149) Union +(148) Union -(150) HashAggregate [codegen id : 240] +(149) HashAggregate [codegen id : 240] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(151) Exchange +(150) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#132] -(152) HashAggregate [codegen id : 241] +(151) HashAggregate [codegen id : 241] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(153) ReusedExchange [Reuses operator id: 84] +(152) ReusedExchange [Reuses operator id: 83] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135] -(154) HashAggregate [codegen id : 280] +(153) HashAggregate [codegen id : 280] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136, count(1)#137] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sales#42, count(1)#137 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] -(155) Filter [codegen id : 280] +(154) Filter [codegen id : 280] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(156) Project [codegen id : 280] +(155) Project [codegen id : 280] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] -(157) ReusedExchange [Reuses operator id: 103] +(156) ReusedExchange [Reuses operator id: 102] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141] -(158) HashAggregate [codegen id : 319] +(157) HashAggregate [codegen id : 319] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142, count(1)#143] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sales#60, count(1)#143 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] -(159) Filter [codegen id : 319] +(158) Filter [codegen id : 319] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(160) Project [codegen id : 319] +(159) Project [codegen id : 319] Output [6]: [catalog AS channel#145, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] -(161) ReusedExchange [Reuses operator id: 122] +(160) ReusedExchange [Reuses operator id: 121] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148] -(162) HashAggregate [codegen id : 358] +(161) HashAggregate [codegen id : 358] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#76, count(1)#150 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] -(163) Filter [codegen id : 358] +(162) Filter [codegen id : 358] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(164) Project [codegen id : 358] +(163) Project [codegen id : 358] Output [6]: [web AS channel#152, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] -(165) Union +(164) Union -(166) HashAggregate [codegen id : 359] +(165) HashAggregate [codegen id : 359] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#153, isEmpty#154, sum#155] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] -(167) Exchange +(166) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#159] -(168) HashAggregate [codegen id : 360] +(167) HashAggregate [codegen id : 360] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#160, sum(number_sales#43)#161] Results [4]: [channel#47, i_brand_id#7, sum(sales#42)#160 AS sum_sales#89, sum(number_sales#43)#161 AS number_sales#90] -(169) HashAggregate [codegen id : 360] +(168) HashAggregate [codegen id : 360] Input [4]: [channel#47, i_brand_id#7, sum_sales#89, number_sales#90] Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] Results [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] -(170) Exchange +(169) Exchange Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] Arguments: hashpartitioning(channel#47, i_brand_id#7, 5), true, [id=#168] -(171) HashAggregate [codegen id : 361] +(170) HashAggregate [codegen id : 361] Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#169, sum(number_sales#90)#170] Results [6]: [channel#47, i_brand_id#7, null AS i_class_id#171, null AS i_category_id#172, sum(sum_sales#89)#169 AS sum(sum_sales)#173, sum(number_sales#90)#170 AS sum(number_sales)#174] -(172) Union +(171) Union -(173) HashAggregate [codegen id : 362] +(172) HashAggregate [codegen id : 362] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(174) Exchange +(173) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#175] -(175) HashAggregate [codegen id : 363] +(174) HashAggregate [codegen id : 363] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(176) ReusedExchange [Reuses operator id: 84] +(175) ReusedExchange [Reuses operator id: 83] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] -(177) HashAggregate [codegen id : 402] +(176) HashAggregate [codegen id : 402] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179, count(1)#180] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sales#42, count(1)#180 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] -(178) Filter [codegen id : 402] +(177) Filter [codegen id : 402] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(179) Project [codegen id : 402] +(178) Project [codegen id : 402] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] -(180) ReusedExchange [Reuses operator id: 103] +(179) ReusedExchange [Reuses operator id: 102] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] -(181) HashAggregate [codegen id : 441] +(180) HashAggregate [codegen id : 441] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185, count(1)#186] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sales#60, count(1)#186 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] -(182) Filter [codegen id : 441] +(181) Filter [codegen id : 441] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(183) Project [codegen id : 441] +(182) Project [codegen id : 441] Output [6]: [catalog AS channel#188, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] -(184) ReusedExchange [Reuses operator id: 122] +(183) ReusedExchange [Reuses operator id: 121] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] -(185) HashAggregate [codegen id : 480] +(184) HashAggregate [codegen id : 480] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#76, count(1)#193 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] -(186) Filter [codegen id : 480] +(185) Filter [codegen id : 480] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(187) Project [codegen id : 480] +(186) Project [codegen id : 480] Output [6]: [web AS channel#195, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] -(188) Union +(187) Union -(189) HashAggregate [codegen id : 481] +(188) HashAggregate [codegen id : 481] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#196, isEmpty#197, sum#198] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] -(190) Exchange +(189) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#202] -(191) HashAggregate [codegen id : 482] +(190) HashAggregate [codegen id : 482] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#203, sum(number_sales#43)#204] Results [3]: [channel#47, sum(sales#42)#203 AS sum_sales#89, sum(number_sales#43)#204 AS number_sales#90] -(192) HashAggregate [codegen id : 482] +(191) HashAggregate [codegen id : 482] Input [3]: [channel#47, sum_sales#89, number_sales#90] Keys [1]: [channel#47] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#205, isEmpty#206, sum#207] Results [4]: [channel#47, sum#208, isEmpty#209, sum#210] -(193) Exchange +(192) Exchange Input [4]: [channel#47, sum#208, isEmpty#209, sum#210] Arguments: hashpartitioning(channel#47, 5), true, [id=#211] -(194) HashAggregate [codegen id : 483] +(193) HashAggregate [codegen id : 483] Input [4]: [channel#47, sum#208, isEmpty#209, sum#210] Keys [1]: [channel#47] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#212, sum(number_sales#90)#213] Results [6]: [channel#47, null AS i_brand_id#214, null AS i_class_id#215, null AS i_category_id#216, sum(sum_sales#89)#212 AS sum(sum_sales)#217, sum(number_sales#90)#213 AS sum(number_sales)#218] -(195) Union +(194) Union -(196) HashAggregate [codegen id : 484] +(195) HashAggregate [codegen id : 484] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(197) Exchange +(196) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#219] -(198) HashAggregate [codegen id : 485] +(197) HashAggregate [codegen id : 485] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(199) ReusedExchange [Reuses operator id: 84] +(198) ReusedExchange [Reuses operator id: 83] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] -(200) HashAggregate [codegen id : 524] +(199) HashAggregate [codegen id : 524] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#42, count(1)#224 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] -(201) Filter [codegen id : 524] +(200) Filter [codegen id : 524] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(202) Project [codegen id : 524] +(201) Project [codegen id : 524] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] -(203) ReusedExchange [Reuses operator id: 103] +(202) ReusedExchange [Reuses operator id: 102] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] -(204) HashAggregate [codegen id : 563] +(203) HashAggregate [codegen id : 563] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229, count(1)#230] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sales#60, count(1)#230 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] -(205) Filter [codegen id : 563] +(204) Filter [codegen id : 563] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(206) Project [codegen id : 563] +(205) Project [codegen id : 563] Output [6]: [catalog AS channel#232, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] -(207) ReusedExchange [Reuses operator id: 122] +(206) ReusedExchange [Reuses operator id: 121] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235] -(208) HashAggregate [codegen id : 602] +(207) HashAggregate [codegen id : 602] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#76, count(1)#237 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] -(209) Filter [codegen id : 602] +(208) Filter [codegen id : 602] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(210) Project [codegen id : 602] +(209) Project [codegen id : 602] Output [6]: [web AS channel#239, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] -(211) Union +(210) Union -(212) HashAggregate [codegen id : 603] +(211) HashAggregate [codegen id : 603] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#240, isEmpty#241, sum#242] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] -(213) Exchange +(212) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#246] -(214) HashAggregate [codegen id : 604] +(213) HashAggregate [codegen id : 604] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#247, sum(number_sales#43)#248] Results [2]: [sum(sales#42)#247 AS sum_sales#89, sum(number_sales#43)#248 AS number_sales#90] -(215) HashAggregate [codegen id : 604] +(214) HashAggregate [codegen id : 604] Input [2]: [sum_sales#89, number_sales#90] Keys: [] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#249, isEmpty#250, sum#251] Results [3]: [sum#252, isEmpty#253, sum#254] -(216) Exchange +(215) Exchange Input [3]: [sum#252, isEmpty#253, sum#254] Arguments: SinglePartition, true, [id=#255] -(217) HashAggregate [codegen id : 605] +(216) HashAggregate [codegen id : 605] Input [3]: [sum#252, isEmpty#253, sum#254] Keys: [] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#256, sum(number_sales#90)#257] Results [6]: [null AS channel#258, null AS i_brand_id#259, null AS i_class_id#260, null AS i_category_id#261, sum(sum_sales#89)#256 AS sum(sum_sales)#262, sum(number_sales#90)#257 AS sum(number_sales)#263] -(218) Union +(217) Union -(219) HashAggregate [codegen id : 606] +(218) HashAggregate [codegen id : 606] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(220) Exchange +(219) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#264] -(221) HashAggregate [codegen id : 607] +(220) HashAggregate [codegen id : 607] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(222) TakeOrderedAndProject +(221) TakeOrderedAndProject Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: 100, [channel#47 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (252) -+- Exchange (251) - +- * HashAggregate (250) - +- Union (249) - :- * Project (232) - : +- * BroadcastHashJoin Inner BuildRight (231) - : :- * Filter (225) - : : +- * ColumnarToRow (224) - : : +- Scan parquet default.store_sales (223) - : +- BroadcastExchange (230) - : +- * Project (229) - : +- * Filter (228) - : +- * ColumnarToRow (227) - : +- Scan parquet default.date_dim (226) - :- * Project (242) - : +- * BroadcastHashJoin Inner BuildRight (241) - : :- * Filter (235) - : : +- * ColumnarToRow (234) - : : +- Scan parquet default.catalog_sales (233) - : +- BroadcastExchange (240) - : +- * Project (239) - : +- * Filter (238) - : +- * ColumnarToRow (237) - : +- Scan parquet default.date_dim (236) - +- * Project (248) - +- * BroadcastHashJoin Inner BuildRight (247) - :- * Filter (245) - : +- * ColumnarToRow (244) - : +- Scan parquet default.web_sales (243) - +- ReusedExchange (246) - - -(223) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 85 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* HashAggregate (251) ++- Exchange (250) + +- * HashAggregate (249) + +- Union (248) + :- * Project (231) + : +- * BroadcastHashJoin Inner BuildRight (230) + : :- * Filter (224) + : : +- * ColumnarToRow (223) + : : +- Scan parquet default.store_sales (222) + : +- BroadcastExchange (229) + : +- * Project (228) + : +- * Filter (227) + : +- * ColumnarToRow (226) + : +- Scan parquet default.date_dim (225) + :- * Project (241) + : +- * BroadcastHashJoin Inner BuildRight (240) + : :- * Filter (234) + : : +- * ColumnarToRow (233) + : : +- Scan parquet default.catalog_sales (232) + : +- BroadcastExchange (239) + : +- * Project (238) + : +- * Filter (237) + : +- * ColumnarToRow (236) + : +- Scan parquet default.date_dim (235) + +- * Project (247) + +- * BroadcastHashJoin Inner BuildRight (246) + :- * Filter (244) + : +- * ColumnarToRow (243) + : +- Scan parquet default.web_sales (242) + +- ReusedExchange (245) + + +(222) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(224) ColumnarToRow [codegen id : 2] +(223) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(225) Filter [codegen id : 2] +(224) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(226) Scan parquet default.date_dim +(225) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(227) ColumnarToRow [codegen id : 1] +(226) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(228) Filter [codegen id : 1] +(227) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(229) Project [codegen id : 1] +(228) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(230) BroadcastExchange +(229) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#265] -(231) BroadcastHashJoin [codegen id : 2] +(230) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(232) Project [codegen id : 2] +(231) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#266, ss_list_price#4 AS list_price#267] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(233) Scan parquet default.catalog_sales +(232) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(234) ColumnarToRow [codegen id : 4] +(233) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] -(235) Filter [codegen id : 4] +(234) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Condition : isnotnull(cs_sold_date_sk#18) -(236) Scan parquet default.date_dim +(235) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(237) ColumnarToRow [codegen id : 3] +(236) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(238) Filter [codegen id : 3] +(237) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(239) Project [codegen id : 3] +(238) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(240) BroadcastExchange +(239) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#268] -(241) BroadcastHashJoin [codegen id : 4] +(240) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(242) Project [codegen id : 4] +(241) Project [codegen id : 4] Output [2]: [cs_quantity#48 AS quantity#269, cs_list_price#49 AS list_price#270] Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(243) Scan parquet default.web_sales +(242) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(244) ColumnarToRow [codegen id : 6] +(243) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -(245) Filter [codegen id : 6] +(244) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Condition : isnotnull(ws_sold_date_sk#22) -(246) ReusedExchange [Reuses operator id: 240] +(245) ReusedExchange [Reuses operator id: 239] Output [1]: [d_date_sk#10] -(247) BroadcastHashJoin [codegen id : 6] +(246) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(248) Project [codegen id : 6] +(247) Project [codegen id : 6] Output [2]: [ws_quantity#64 AS quantity#271, ws_list_price#65 AS list_price#272] Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(249) Union +(248) Union -(250) HashAggregate [codegen id : 7] +(249) HashAggregate [codegen id : 7] Input [2]: [quantity#266, list_price#267] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#273, count#274] Results [2]: [sum#275, count#276] -(251) Exchange +(250) Exchange Input [2]: [sum#275, count#276] Arguments: SinglePartition, true, [id=#277] -(252) HashAggregate [codegen id : 8] +(251) HashAggregate [codegen id : 8] Input [2]: [sum#275, count#276] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278 AS average_sales#279] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:4 Hosting operator id = 131 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:5 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:6 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:7 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:7 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:8 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:8 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:9 Hosting operator id = 163 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:10 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:11 Hosting operator id = 182 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:11 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:12 Hosting operator id = 186 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:12 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:13 Hosting operator id = 201 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:13 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:14 Hosting operator id = 205 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:14 Hosting operator id = 204 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:15 Hosting operator id = 209 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:15 Hosting operator id = 208 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] 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 30856e02f2b62..bec87902b9d0d 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 @@ -126,81 +126,80 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter + InputAdapter + Exchange [brand_id,class_id,category_id] #10 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #11 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [i_item_sk,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] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #11 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (13) + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (9) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #12 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + BroadcastExchange #15 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #17 WholeStageCodegen (19) 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 238053a3428e3..bce3b090bc52b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,210 +1,209 @@ == Physical Plan == -TakeOrderedAndProject (206) -+- * HashAggregate (205) - +- Exchange (204) - +- * HashAggregate (203) - +- Union (202) - :- * HashAggregate (182) - : +- Exchange (181) - : +- * HashAggregate (180) - : +- Union (179) - : :- * HashAggregate (159) - : : +- Exchange (158) - : : +- * HashAggregate (157) - : : +- Union (156) - : : :- * HashAggregate (136) - : : : +- Exchange (135) - : : : +- * HashAggregate (134) - : : : +- Union (133) - : : : :- * HashAggregate (113) - : : : : +- Exchange (112) - : : : : +- * HashAggregate (111) - : : : : +- Union (110) - : : : : :- * Project (77) - : : : : : +- * Filter (76) - : : : : : +- * HashAggregate (75) - : : : : : +- Exchange (74) - : : : : : +- * HashAggregate (73) - : : : : : +- * Project (72) - : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : :- * Project (65) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (205) ++- * HashAggregate (204) + +- Exchange (203) + +- * HashAggregate (202) + +- Union (201) + :- * HashAggregate (181) + : +- Exchange (180) + : +- * HashAggregate (179) + : +- Union (178) + : :- * HashAggregate (158) + : : +- Exchange (157) + : : +- * HashAggregate (156) + : : +- Union (155) + : : :- * HashAggregate (135) + : : : +- Exchange (134) + : : : +- * HashAggregate (133) + : : : +- Union (132) + : : : :- * HashAggregate (112) + : : : : +- Exchange (111) + : : : : +- * HashAggregate (110) + : : : : +- Union (109) + : : : : :- * Project (76) + : : : : : +- * Filter (75) + : : : : : +- * HashAggregate (74) + : : : : : +- Exchange (73) + : : : : : +- * HashAggregate (72) + : : : : : +- * Project (71) + : : : : : +- * BroadcastHashJoin Inner BuildRight (70) + : : : : : :- * Project (64) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (56) : : : : : : : :- * Filter (3) : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : : +- BroadcastExchange (56) - : : : : : : : +- * Project (55) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : +- BroadcastExchange (55) + : : : : : : : +- * Project (54) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) : : : : : : : :- * Filter (6) : : : : : : : : +- * ColumnarToRow (5) : : : : : : : : +- Scan parquet default.item (4) - : : : : : : : +- BroadcastExchange (53) - : : : : : : : +- * HashAggregate (52) - : : : : : : : +- * HashAggregate (51) - : : : : : : : +- * HashAggregate (50) - : : : : : : : +- Exchange (49) - : : : : : : : +- * HashAggregate (48) - : : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : : : : : :- * Project (22) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : : : : : :- * Project (15) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : : : :- * Filter (9) - : : : : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : : : +- * Filter (12) - : : : : : : : : : : +- * ColumnarToRow (11) - : : : : : : : : : : +- Scan parquet default.item (10) - : : : : : : : : : +- BroadcastExchange (20) - : : : : : : : : : +- * Project (19) - : : : : : : : : : +- * Filter (18) - : : : : : : : : : +- * ColumnarToRow (17) - : : : : : : : : : +- Scan parquet default.date_dim (16) - : : : : : : : : +- BroadcastExchange (35) - : : : : : : : : +- * Project (34) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : :- * Project (31) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : : : : : :- * Filter (25) - : : : : : : : : : : +- * ColumnarToRow (24) - : : : : : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : : : : : +- BroadcastExchange (29) - : : : : : : : : : +- * Filter (28) - : : : : : : : : : +- * ColumnarToRow (27) - : : : : : : : : : +- Scan parquet default.item (26) - : : : : : : : : +- ReusedExchange (32) - : : : : : : : +- BroadcastExchange (46) - : : : : : : : +- * Project (45) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : : : : :- * Project (42) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : :- * Filter (39) - : : : : : : : : : +- * ColumnarToRow (38) - : : : : : : : : : +- Scan parquet default.web_sales (37) - : : : : : : : : +- ReusedExchange (40) - : : : : : : : +- ReusedExchange (43) - : : : : : : +- BroadcastExchange (63) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : : : : : :- * Filter (60) - : : : : : : : +- * ColumnarToRow (59) - : : : : : : : +- Scan parquet default.item (58) - : : : : : : +- ReusedExchange (61) - : : : : : +- BroadcastExchange (70) - : : : : : +- * Project (69) - : : : : : +- * Filter (68) - : : : : : +- * ColumnarToRow (67) - : : : : : +- Scan parquet default.date_dim (66) - : : : : :- * Project (93) - : : : : : +- * Filter (92) - : : : : : +- * HashAggregate (91) - : : : : : +- Exchange (90) - : : : : : +- * HashAggregate (89) - : : : : : +- * Project (88) - : : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : : :- * Project (85) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : : : : : :- * Filter (80) - : : : : : : : : +- * ColumnarToRow (79) - : : : : : : : : +- Scan parquet default.catalog_sales (78) - : : : : : : : +- ReusedExchange (81) - : : : : : : +- ReusedExchange (83) - : : : : : +- ReusedExchange (86) - : : : : +- * Project (109) - : : : : +- * Filter (108) - : : : : +- * HashAggregate (107) - : : : : +- Exchange (106) - : : : : +- * HashAggregate (105) - : : : : +- * Project (104) - : : : : +- * BroadcastHashJoin Inner BuildRight (103) - : : : : :- * Project (101) - : : : : : +- * BroadcastHashJoin Inner BuildRight (100) - : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : : : : : :- * Filter (96) - : : : : : : : +- * ColumnarToRow (95) - : : : : : : : +- Scan parquet default.web_sales (94) - : : : : : : +- ReusedExchange (97) - : : : : : +- ReusedExchange (99) - : : : : +- ReusedExchange (102) - : : : +- * HashAggregate (132) - : : : +- Exchange (131) - : : : +- * HashAggregate (130) - : : : +- * HashAggregate (129) - : : : +- Exchange (128) - : : : +- * HashAggregate (127) - : : : +- Union (126) - : : : :- * Project (117) - : : : : +- * Filter (116) - : : : : +- * HashAggregate (115) - : : : : +- ReusedExchange (114) - : : : :- * Project (121) - : : : : +- * Filter (120) - : : : : +- * HashAggregate (119) - : : : : +- ReusedExchange (118) - : : : +- * Project (125) - : : : +- * Filter (124) - : : : +- * HashAggregate (123) - : : : +- ReusedExchange (122) - : : +- * HashAggregate (155) - : : +- Exchange (154) - : : +- * HashAggregate (153) - : : +- * HashAggregate (152) - : : +- Exchange (151) - : : +- * HashAggregate (150) - : : +- Union (149) - : : :- * Project (140) - : : : +- * Filter (139) - : : : +- * HashAggregate (138) - : : : +- ReusedExchange (137) - : : :- * Project (144) - : : : +- * Filter (143) - : : : +- * HashAggregate (142) - : : : +- ReusedExchange (141) - : : +- * Project (148) - : : +- * Filter (147) - : : +- * HashAggregate (146) - : : +- ReusedExchange (145) - : +- * HashAggregate (178) - : +- Exchange (177) - : +- * HashAggregate (176) - : +- * HashAggregate (175) - : +- Exchange (174) - : +- * HashAggregate (173) - : +- Union (172) - : :- * Project (163) - : : +- * Filter (162) - : : +- * HashAggregate (161) - : : +- ReusedExchange (160) - : :- * Project (167) - : : +- * Filter (166) - : : +- * HashAggregate (165) - : : +- ReusedExchange (164) - : +- * Project (171) - : +- * Filter (170) - : +- * HashAggregate (169) - : +- ReusedExchange (168) - +- * HashAggregate (201) - +- Exchange (200) - +- * HashAggregate (199) - +- * HashAggregate (198) - +- Exchange (197) - +- * HashAggregate (196) - +- Union (195) - :- * Project (186) - : +- * Filter (185) - : +- * HashAggregate (184) - : +- ReusedExchange (183) - :- * Project (190) - : +- * Filter (189) - : +- * HashAggregate (188) - : +- ReusedExchange (187) - +- * Project (194) - +- * Filter (193) - +- * HashAggregate (192) - +- ReusedExchange (191) + : : : : : : : +- BroadcastExchange (52) + : : : : : : : +- * HashAggregate (51) + : : : : : : : +- * HashAggregate (50) + : : : : : : : +- Exchange (49) + : : : : : : : +- * HashAggregate (48) + : : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : : : : : :- * Project (22) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : : : : : :- * Project (15) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : : : :- * Filter (9) + : : : : : : : : : : : +- * ColumnarToRow (8) + : : : : : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : : : +- * Filter (12) + : : : : : : : : : : +- * ColumnarToRow (11) + : : : : : : : : : : +- Scan parquet default.item (10) + : : : : : : : : : +- BroadcastExchange (20) + : : : : : : : : : +- * Project (19) + : : : : : : : : : +- * Filter (18) + : : : : : : : : : +- * ColumnarToRow (17) + : : : : : : : : : +- Scan parquet default.date_dim (16) + : : : : : : : : +- BroadcastExchange (35) + : : : : : : : : +- * Project (34) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : : : : :- * Project (31) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : : : : : :- * Filter (25) + : : : : : : : : : : +- * ColumnarToRow (24) + : : : : : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : : : : : +- BroadcastExchange (29) + : : : : : : : : : +- * Filter (28) + : : : : : : : : : +- * ColumnarToRow (27) + : : : : : : : : : +- Scan parquet default.item (26) + : : : : : : : : +- ReusedExchange (32) + : : : : : : : +- BroadcastExchange (46) + : : : : : : : +- * Project (45) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : : : : :- * Project (42) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : :- * Filter (39) + : : : : : : : : : +- * ColumnarToRow (38) + : : : : : : : : : +- Scan parquet default.web_sales (37) + : : : : : : : : +- ReusedExchange (40) + : : : : : : : +- ReusedExchange (43) + : : : : : : +- BroadcastExchange (62) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (61) + : : : : : : :- * Filter (59) + : : : : : : : +- * ColumnarToRow (58) + : : : : : : : +- Scan parquet default.item (57) + : : : : : : +- ReusedExchange (60) + : : : : : +- BroadcastExchange (69) + : : : : : +- * Project (68) + : : : : : +- * Filter (67) + : : : : : +- * ColumnarToRow (66) + : : : : : +- Scan parquet default.date_dim (65) + : : : : :- * Project (92) + : : : : : +- * Filter (91) + : : : : : +- * HashAggregate (90) + : : : : : +- Exchange (89) + : : : : : +- * HashAggregate (88) + : : : : : +- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (84) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (83) + : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : : : : : : :- * Filter (79) + : : : : : : : : +- * ColumnarToRow (78) + : : : : : : : : +- Scan parquet default.catalog_sales (77) + : : : : : : : +- ReusedExchange (80) + : : : : : : +- ReusedExchange (82) + : : : : : +- ReusedExchange (85) + : : : : +- * Project (108) + : : : : +- * Filter (107) + : : : : +- * HashAggregate (106) + : : : : +- Exchange (105) + : : : : +- * HashAggregate (104) + : : : : +- * Project (103) + : : : : +- * BroadcastHashJoin Inner BuildRight (102) + : : : : :- * Project (100) + : : : : : +- * BroadcastHashJoin Inner BuildRight (99) + : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (97) + : : : : : : :- * Filter (95) + : : : : : : : +- * ColumnarToRow (94) + : : : : : : : +- Scan parquet default.web_sales (93) + : : : : : : +- ReusedExchange (96) + : : : : : +- ReusedExchange (98) + : : : : +- ReusedExchange (101) + : : : +- * HashAggregate (131) + : : : +- Exchange (130) + : : : +- * HashAggregate (129) + : : : +- * HashAggregate (128) + : : : +- Exchange (127) + : : : +- * HashAggregate (126) + : : : +- Union (125) + : : : :- * Project (116) + : : : : +- * Filter (115) + : : : : +- * HashAggregate (114) + : : : : +- ReusedExchange (113) + : : : :- * Project (120) + : : : : +- * Filter (119) + : : : : +- * HashAggregate (118) + : : : : +- ReusedExchange (117) + : : : +- * Project (124) + : : : +- * Filter (123) + : : : +- * HashAggregate (122) + : : : +- ReusedExchange (121) + : : +- * HashAggregate (154) + : : +- Exchange (153) + : : +- * HashAggregate (152) + : : +- * HashAggregate (151) + : : +- Exchange (150) + : : +- * HashAggregate (149) + : : +- Union (148) + : : :- * Project (139) + : : : +- * Filter (138) + : : : +- * HashAggregate (137) + : : : +- ReusedExchange (136) + : : :- * Project (143) + : : : +- * Filter (142) + : : : +- * HashAggregate (141) + : : : +- ReusedExchange (140) + : : +- * Project (147) + : : +- * Filter (146) + : : +- * HashAggregate (145) + : : +- ReusedExchange (144) + : +- * HashAggregate (177) + : +- Exchange (176) + : +- * HashAggregate (175) + : +- * HashAggregate (174) + : +- Exchange (173) + : +- * HashAggregate (172) + : +- Union (171) + : :- * Project (162) + : : +- * Filter (161) + : : +- * HashAggregate (160) + : : +- ReusedExchange (159) + : :- * Project (166) + : : +- * Filter (165) + : : +- * HashAggregate (164) + : : +- ReusedExchange (163) + : +- * Project (170) + : +- * Filter (169) + : +- * HashAggregate (168) + : +- ReusedExchange (167) + +- * HashAggregate (200) + +- Exchange (199) + +- * HashAggregate (198) + +- * HashAggregate (197) + +- Exchange (196) + +- * HashAggregate (195) + +- Union (194) + :- * Project (185) + : +- * Filter (184) + : +- * HashAggregate (183) + : +- ReusedExchange (182) + :- * Project (189) + : +- * Filter (188) + : +- * HashAggregate (187) + : +- ReusedExchange (186) + +- * Project (193) + +- * Filter (192) + +- * HashAggregate (191) + +- ReusedExchange (190) (1) Scan parquet default.store_sales @@ -441,940 +440,933 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(52) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(53) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(54) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(55) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(56) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(57) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(58) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(59) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : isnotnull(i_item_sk#5) -(61) ReusedExchange [Reuses operator id: 56] +(60) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(61) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(62) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(63) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(64) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(65) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#28] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(66) ColumnarToRow [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(68) Filter [codegen id : 24] +(67) Filter [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#28)) AND (d_year#11 = 2000)) AND (d_moy#28 = 11)) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(68) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(70) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] -(71) BroadcastHashJoin [codegen id : 25] +(70) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(71) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(72) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#30, isEmpty#31, count#32] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -(74) Exchange +(73) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#36] -(75) HashAggregate [codegen id : 26] +(74) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(76) Filter [codegen id : 26] +(75) Filter [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(77) Project [codegen id : 26] +(76) Project [codegen id : 26] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(78) Scan parquet default.catalog_sales +(77) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 51] +(78) ColumnarToRow [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -(80) Filter [codegen id : 51] +(79) Filter [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) -(81) ReusedExchange [Reuses operator id: 56] +(80) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 51] +(81) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(82) ReusedExchange [Reuses operator id: 62] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(84) BroadcastHashJoin [codegen id : 51] +(83) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [i_item_sk#5] Join condition: None -(85) Project [codegen id : 51] +(84) Project [codegen id : 51] Output [6]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(86) ReusedExchange [Reuses operator id: 70] +(85) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#10] -(87) BroadcastHashJoin [codegen id : 51] +(86) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(88) Project [codegen id : 51] +(87) Project [codegen id : 51] Output [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(89) HashAggregate [codegen id : 51] +(88) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -(90) Exchange +(89) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#53] -(91) HashAggregate [codegen id : 52] +(90) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(92) Filter [codegen id : 52] +(91) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(93) Project [codegen id : 52] +(92) Project [codegen id : 52] Output [6]: [catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(94) Scan parquet default.web_sales +(93) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 77] +(94) ColumnarToRow [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -(96) Filter [codegen id : 77] +(95) Filter [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) -(97) ReusedExchange [Reuses operator id: 56] +(96) ReusedExchange [Reuses operator id: 55] Output [1]: [ss_item_sk#25] -(98) BroadcastHashJoin [codegen id : 77] +(97) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [ss_item_sk#25] Join condition: None -(99) ReusedExchange [Reuses operator id: 63] +(98) ReusedExchange [Reuses operator id: 62] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(100) BroadcastHashJoin [codegen id : 77] +(99) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [i_item_sk#5] Join condition: None -(101) Project [codegen id : 77] +(100) Project [codegen id : 77] Output [6]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(102) ReusedExchange [Reuses operator id: 70] +(101) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#10] -(103) BroadcastHashJoin [codegen id : 77] +(102) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(104) Project [codegen id : 77] +(103) Project [codegen id : 77] Output [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(105) HashAggregate [codegen id : 77] +(104) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#62, isEmpty#63, count#64] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -(106) Exchange +(105) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#68] -(107) HashAggregate [codegen id : 78] +(106) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(108) Filter [codegen id : 78] +(107) Filter [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(109) Project [codegen id : 78] +(108) Project [codegen id : 78] Output [6]: [web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(110) Union +(109) Union -(111) HashAggregate [codegen id : 79] +(110) HashAggregate [codegen id : 79] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#75, isEmpty#76, sum#77] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] -(112) Exchange +(111) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#81] -(113) HashAggregate [codegen id : 80] +(112) HashAggregate [codegen id : 80] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#82, sum(number_sales#40)#83] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#39)#82 AS sum_sales#84, sum(number_sales#40)#83 AS number_sales#85] -(114) ReusedExchange [Reuses operator id: 74] +(113) ReusedExchange [Reuses operator id: 73] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] -(115) HashAggregate [codegen id : 106] +(114) HashAggregate [codegen id : 106] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89, count(1)#90] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#39, count(1)#90 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -(116) Filter [codegen id : 106] +(115) Filter [codegen id : 106] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(117) Project [codegen id : 106] +(116) Project [codegen id : 106] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -(118) ReusedExchange [Reuses operator id: 90] +(117) ReusedExchange [Reuses operator id: 89] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] -(119) HashAggregate [codegen id : 132] +(118) HashAggregate [codegen id : 132] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95, count(1)#96] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#56, count(1)#96 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -(120) Filter [codegen id : 132] +(119) Filter [codegen id : 132] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(121) Project [codegen id : 132] +(120) Project [codegen id : 132] Output [6]: [catalog AS channel#98, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -(122) ReusedExchange [Reuses operator id: 106] +(121) ReusedExchange [Reuses operator id: 105] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] -(123) HashAggregate [codegen id : 158] +(122) HashAggregate [codegen id : 158] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#71, count(1)#103 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] -(124) Filter [codegen id : 158] +(123) Filter [codegen id : 158] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(125) Project [codegen id : 158] +(124) Project [codegen id : 158] Output [6]: [web AS channel#105, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] -(126) Union +(125) Union -(127) HashAggregate [codegen id : 159] +(126) HashAggregate [codegen id : 159] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] -(128) Exchange +(127) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#112] -(129) HashAggregate [codegen id : 160] +(128) HashAggregate [codegen id : 160] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#113, sum(number_sales#40)#114] Results [5]: [channel#44, i_brand_id#6, i_class_id#7, sum(sales#39)#113 AS sum_sales#84, sum(number_sales#40)#114 AS number_sales#85] -(130) HashAggregate [codegen id : 160] +(129) HashAggregate [codegen id : 160] Input [5]: [channel#44, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85] Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] -(131) Exchange +(130) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, 5), true, [id=#121] -(132) HashAggregate [codegen id : 161] +(131) HashAggregate [codegen id : 161] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#122, sum(number_sales#85)#123] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, null AS i_category_id#124, sum(sum_sales#84)#122 AS sum(sum_sales)#125, sum(number_sales#85)#123 AS sum(number_sales)#126] -(133) Union +(132) Union -(134) HashAggregate [codegen id : 162] +(133) HashAggregate [codegen id : 162] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(135) Exchange +(134) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#127] -(136) HashAggregate [codegen id : 163] +(135) HashAggregate [codegen id : 163] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(137) ReusedExchange [Reuses operator id: 74] +(136) ReusedExchange [Reuses operator id: 73] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130] -(138) HashAggregate [codegen id : 189] +(137) HashAggregate [codegen id : 189] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131, count(1)#132] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sales#39, count(1)#132 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] -(139) Filter [codegen id : 189] +(138) Filter [codegen id : 189] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(140) Project [codegen id : 189] +(139) Project [codegen id : 189] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] -(141) ReusedExchange [Reuses operator id: 90] +(140) ReusedExchange [Reuses operator id: 89] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136] -(142) HashAggregate [codegen id : 215] +(141) HashAggregate [codegen id : 215] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137, count(1)#138] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sales#56, count(1)#138 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] -(143) Filter [codegen id : 215] +(142) Filter [codegen id : 215] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(144) Project [codegen id : 215] +(143) Project [codegen id : 215] Output [6]: [catalog AS channel#140, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] -(145) ReusedExchange [Reuses operator id: 106] +(144) ReusedExchange [Reuses operator id: 105] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143] -(146) HashAggregate [codegen id : 241] +(145) HashAggregate [codegen id : 241] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144, count(1)#145] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sales#71, count(1)#145 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] -(147) Filter [codegen id : 241] +(146) Filter [codegen id : 241] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(148) Project [codegen id : 241] +(147) Project [codegen id : 241] Output [6]: [web AS channel#147, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] -(149) Union +(148) Union -(150) HashAggregate [codegen id : 242] +(149) HashAggregate [codegen id : 242] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] -(151) Exchange +(150) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#154] -(152) HashAggregate [codegen id : 243] +(151) HashAggregate [codegen id : 243] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#155, sum(number_sales#40)#156] Results [4]: [channel#44, i_brand_id#6, sum(sales#39)#155 AS sum_sales#84, sum(number_sales#40)#156 AS number_sales#85] -(153) HashAggregate [codegen id : 243] +(152) HashAggregate [codegen id : 243] Input [4]: [channel#44, i_brand_id#6, sum_sales#84, number_sales#85] Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] Results [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] -(154) Exchange +(153) Exchange Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] Arguments: hashpartitioning(channel#44, i_brand_id#6, 5), true, [id=#163] -(155) HashAggregate [codegen id : 244] +(154) HashAggregate [codegen id : 244] Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#164, sum(number_sales#85)#165] Results [6]: [channel#44, i_brand_id#6, null AS i_class_id#166, null AS i_category_id#167, sum(sum_sales#84)#164 AS sum(sum_sales)#168, sum(number_sales#85)#165 AS sum(number_sales)#169] -(156) Union +(155) Union -(157) HashAggregate [codegen id : 245] +(156) HashAggregate [codegen id : 245] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(158) Exchange +(157) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#170] -(159) HashAggregate [codegen id : 246] +(158) HashAggregate [codegen id : 246] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(160) ReusedExchange [Reuses operator id: 74] +(159) ReusedExchange [Reuses operator id: 73] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173] -(161) HashAggregate [codegen id : 272] +(160) HashAggregate [codegen id : 272] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#39, count(1)#175 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] -(162) Filter [codegen id : 272] +(161) Filter [codegen id : 272] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(163) Project [codegen id : 272] +(162) Project [codegen id : 272] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] -(164) ReusedExchange [Reuses operator id: 90] +(163) ReusedExchange [Reuses operator id: 89] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179] -(165) HashAggregate [codegen id : 298] +(164) HashAggregate [codegen id : 298] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#56, count(1)#181 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] -(166) Filter [codegen id : 298] +(165) Filter [codegen id : 298] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(167) Project [codegen id : 298] +(166) Project [codegen id : 298] Output [6]: [catalog AS channel#183, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] -(168) ReusedExchange [Reuses operator id: 106] +(167) ReusedExchange [Reuses operator id: 105] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186] -(169) HashAggregate [codegen id : 324] +(168) HashAggregate [codegen id : 324] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#71, count(1)#188 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] -(170) Filter [codegen id : 324] +(169) Filter [codegen id : 324] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(171) Project [codegen id : 324] +(170) Project [codegen id : 324] Output [6]: [web AS channel#190, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] -(172) Union +(171) Union -(173) HashAggregate [codegen id : 325] +(172) HashAggregate [codegen id : 325] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] -(174) Exchange +(173) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#197] -(175) HashAggregate [codegen id : 326] +(174) HashAggregate [codegen id : 326] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#198, sum(number_sales#40)#199] Results [3]: [channel#44, sum(sales#39)#198 AS sum_sales#84, sum(number_sales#40)#199 AS number_sales#85] -(176) HashAggregate [codegen id : 326] +(175) HashAggregate [codegen id : 326] Input [3]: [channel#44, sum_sales#84, number_sales#85] Keys [1]: [channel#44] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] Results [4]: [channel#44, sum#203, isEmpty#204, sum#205] -(177) Exchange +(176) Exchange Input [4]: [channel#44, sum#203, isEmpty#204, sum#205] Arguments: hashpartitioning(channel#44, 5), true, [id=#206] -(178) HashAggregate [codegen id : 327] +(177) HashAggregate [codegen id : 327] Input [4]: [channel#44, sum#203, isEmpty#204, sum#205] Keys [1]: [channel#44] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#207, sum(number_sales#85)#208] Results [6]: [channel#44, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#84)#207 AS sum(sum_sales)#212, sum(number_sales#85)#208 AS sum(number_sales)#213] -(179) Union +(178) Union -(180) HashAggregate [codegen id : 328] +(179) HashAggregate [codegen id : 328] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(181) Exchange +(180) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#214] -(182) HashAggregate [codegen id : 329] +(181) HashAggregate [codegen id : 329] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(183) ReusedExchange [Reuses operator id: 74] +(182) ReusedExchange [Reuses operator id: 73] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] -(184) HashAggregate [codegen id : 355] +(183) HashAggregate [codegen id : 355] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218, count(1)#219] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sales#39, count(1)#219 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] -(185) Filter [codegen id : 355] +(184) Filter [codegen id : 355] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(186) Project [codegen id : 355] +(185) Project [codegen id : 355] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] -(187) ReusedExchange [Reuses operator id: 90] +(186) ReusedExchange [Reuses operator id: 89] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] -(188) HashAggregate [codegen id : 381] +(187) HashAggregate [codegen id : 381] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#56, count(1)#225 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] -(189) Filter [codegen id : 381] +(188) Filter [codegen id : 381] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(190) Project [codegen id : 381] +(189) Project [codegen id : 381] Output [6]: [catalog AS channel#227, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] -(191) ReusedExchange [Reuses operator id: 106] +(190) ReusedExchange [Reuses operator id: 105] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230] -(192) HashAggregate [codegen id : 407] +(191) HashAggregate [codegen id : 407] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231, count(1)#232] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sales#71, count(1)#232 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] -(193) Filter [codegen id : 407] +(192) Filter [codegen id : 407] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(194) Project [codegen id : 407] +(193) Project [codegen id : 407] Output [6]: [web AS channel#234, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] -(195) Union +(194) Union -(196) HashAggregate [codegen id : 408] +(195) HashAggregate [codegen id : 408] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#235, isEmpty#236, sum#237] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] -(197) Exchange +(196) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#241] -(198) HashAggregate [codegen id : 409] +(197) HashAggregate [codegen id : 409] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#242, sum(number_sales#40)#243] Results [2]: [sum(sales#39)#242 AS sum_sales#84, sum(number_sales#40)#243 AS number_sales#85] -(199) HashAggregate [codegen id : 409] +(198) HashAggregate [codegen id : 409] Input [2]: [sum_sales#84, number_sales#85] Keys: [] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#244, isEmpty#245, sum#246] Results [3]: [sum#247, isEmpty#248, sum#249] -(200) Exchange +(199) Exchange Input [3]: [sum#247, isEmpty#248, sum#249] Arguments: SinglePartition, true, [id=#250] -(201) HashAggregate [codegen id : 410] +(200) HashAggregate [codegen id : 410] Input [3]: [sum#247, isEmpty#248, sum#249] Keys: [] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#251, sum(number_sales#85)#252] Results [6]: [null AS channel#253, null AS i_brand_id#254, null AS i_class_id#255, null AS i_category_id#256, sum(sum_sales#84)#251 AS sum(sum_sales)#257, sum(number_sales#85)#252 AS sum(number_sales)#258] -(202) Union +(201) Union -(203) HashAggregate [codegen id : 411] +(202) HashAggregate [codegen id : 411] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(204) Exchange +(203) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#259] -(205) HashAggregate [codegen id : 412] +(204) HashAggregate [codegen id : 412] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(206) TakeOrderedAndProject +(205) TakeOrderedAndProject Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (236) -+- Exchange (235) - +- * HashAggregate (234) - +- Union (233) - :- * Project (216) - : +- * BroadcastHashJoin Inner BuildRight (215) - : :- * Filter (209) - : : +- * ColumnarToRow (208) - : : +- Scan parquet default.store_sales (207) - : +- BroadcastExchange (214) - : +- * Project (213) - : +- * Filter (212) - : +- * ColumnarToRow (211) - : +- Scan parquet default.date_dim (210) - :- * Project (226) - : +- * BroadcastHashJoin Inner BuildRight (225) - : :- * Filter (219) - : : +- * ColumnarToRow (218) - : : +- Scan parquet default.catalog_sales (217) - : +- BroadcastExchange (224) - : +- * Project (223) - : +- * Filter (222) - : +- * ColumnarToRow (221) - : +- Scan parquet default.date_dim (220) - +- * Project (232) - +- * BroadcastHashJoin Inner BuildRight (231) - :- * Filter (229) - : +- * ColumnarToRow (228) - : +- Scan parquet default.web_sales (227) - +- ReusedExchange (230) - - -(207) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 75 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (235) ++- Exchange (234) + +- * HashAggregate (233) + +- Union (232) + :- * Project (215) + : +- * BroadcastHashJoin Inner BuildRight (214) + : :- * Filter (208) + : : +- * ColumnarToRow (207) + : : +- Scan parquet default.store_sales (206) + : +- BroadcastExchange (213) + : +- * Project (212) + : +- * Filter (211) + : +- * ColumnarToRow (210) + : +- Scan parquet default.date_dim (209) + :- * Project (225) + : +- * BroadcastHashJoin Inner BuildRight (224) + : :- * Filter (218) + : : +- * ColumnarToRow (217) + : : +- Scan parquet default.catalog_sales (216) + : +- BroadcastExchange (223) + : +- * Project (222) + : +- * Filter (221) + : +- * ColumnarToRow (220) + : +- Scan parquet default.date_dim (219) + +- * Project (231) + +- * BroadcastHashJoin Inner BuildRight (230) + :- * Filter (228) + : +- * ColumnarToRow (227) + : +- Scan parquet default.web_sales (226) + +- ReusedExchange (229) + + +(206) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(208) ColumnarToRow [codegen id : 2] +(207) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(209) Filter [codegen id : 2] +(208) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(210) Scan parquet default.date_dim +(209) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(211) ColumnarToRow [codegen id : 1] +(210) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(212) Filter [codegen id : 1] +(211) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(213) Project [codegen id : 1] +(212) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(214) BroadcastExchange +(213) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] -(215) BroadcastHashJoin [codegen id : 2] +(214) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(216) Project [codegen id : 2] +(215) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#261, ss_list_price#4 AS list_price#262] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(217) Scan parquet default.catalog_sales +(216) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(218) ColumnarToRow [codegen id : 4] +(217) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -(219) Filter [codegen id : 4] +(218) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Condition : isnotnull(cs_sold_date_sk#16) -(220) Scan parquet default.date_dim +(219) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(221) ColumnarToRow [codegen id : 3] +(220) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(222) Filter [codegen id : 3] +(221) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(223) Project [codegen id : 3] +(222) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(224) BroadcastExchange +(223) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#263] -(225) BroadcastHashJoin [codegen id : 4] +(224) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(226) Project [codegen id : 4] +(225) Project [codegen id : 4] Output [2]: [cs_quantity#45 AS quantity#264, cs_list_price#46 AS list_price#265] Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] -(227) Scan parquet default.web_sales +(226) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(228) ColumnarToRow [codegen id : 6] +(227) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -(229) Filter [codegen id : 6] +(228) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Condition : isnotnull(ws_sold_date_sk#20) -(230) ReusedExchange [Reuses operator id: 224] +(229) ReusedExchange [Reuses operator id: 223] Output [1]: [d_date_sk#10] -(231) BroadcastHashJoin [codegen id : 6] +(230) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(232) Project [codegen id : 6] +(231) Project [codegen id : 6] Output [2]: [ws_quantity#60 AS quantity#266, ws_list_price#61 AS list_price#267] Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] -(233) Union +(232) Union -(234) HashAggregate [codegen id : 7] +(233) HashAggregate [codegen id : 7] Input [2]: [quantity#261, list_price#262] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#268, count#269] Results [2]: [sum#270, count#271] -(235) Exchange +(234) Exchange Input [2]: [sum#270, count#271] Arguments: SinglePartition, true, [id=#272] -(236) HashAggregate [codegen id : 8] +(235) HashAggregate [codegen id : 8] Input [2]: [sum#270, count#271] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273 AS average_sales#274] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 91 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:3 Hosting operator id = 107 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:4 Hosting operator id = 115 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:5 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:6 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:7 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:7 Hosting operator id = 138 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:8 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:8 Hosting operator id = 142 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:9 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:9 Hosting operator id = 146 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:10 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:10 Hosting operator id = 161 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:11 Hosting operator id = 166 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:11 Hosting operator id = 165 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:12 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:12 Hosting operator id = 169 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:13 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:13 Hosting operator id = 184 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:14 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:14 Hosting operator id = 188 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:15 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:15 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] 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 e96f1d6fed14f..c01d2868fb63b 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 @@ -118,71 +118,70 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [i_item_sk,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] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #11 + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #14 - WholeStageCodegen (8) + BroadcastExchange #12 + WholeStageCodegen (5) Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + BroadcastExchange #13 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #11 InputAdapter BroadcastExchange #15 WholeStageCodegen (23) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt index 0234a65ac06a5..a6dad5d9b095f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt @@ -1,56 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- Union (51) - :- * HashAggregate (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * SortMergeJoin Inner (24) - : :- * Sort (18) - : : +- Exchange (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.warehouse (4) - : : +- BroadcastExchange (14) - : : +- * Project (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.date_dim (10) - : +- * Sort (23) - : +- Exchange (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.item (19) - :- * HashAggregate (35) - : +- Exchange (34) - : +- * HashAggregate (33) - : +- * HashAggregate (32) - : +- ReusedExchange (31) - :- * HashAggregate (40) - : +- Exchange (39) - : +- * HashAggregate (38) - : +- * HashAggregate (37) - : +- ReusedExchange (36) - :- * HashAggregate (45) - : +- Exchange (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- ReusedExchange (41) - +- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- ReusedExchange (46) +TakeOrderedAndProject (51) ++- Union (50) + :- * HashAggregate (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.warehouse (4) + : : +- BroadcastExchange (14) + : : +- * Project (13) + : : +- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet default.date_dim (10) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) + :- * HashAggregate (34) + : +- Exchange (33) + : +- * HashAggregate (32) + : +- * HashAggregate (31) + : +- ReusedExchange (30) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * HashAggregate (36) + : +- ReusedExchange (35) + :- * HashAggregate (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- * HashAggregate (41) + : +- ReusedExchange (40) + +- * HashAggregate (49) + +- Exchange (48) + +- * HashAggregate (47) + +- * HashAggregate (46) + +- ReusedExchange (45) (1) Scan parquet default.inventory @@ -185,132 +184,125 @@ Results [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, avg(cast (29) HashAggregate [codegen id : 8] Input [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#23] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#26, count#27] - -(30) HashAggregate [codegen id : 8] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#26, count#27] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#28] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, avg(qoh#23)#28 AS qoh#29] +Aggregate Attributes [1]: [avg(qoh#23)#24] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, avg(qoh#23)#24 AS qoh#25] -(31) ReusedExchange [Reuses operator id: 27] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#30, count#31] +(30) ReusedExchange [Reuses operator id: 27] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#26, count#27] -(32) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#30, count#31] +(31) HashAggregate [codegen id : 16] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#26, count#27] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#32] -Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(cast(inv_quantity_on_hand#4 as bigint))#32 AS qoh#23] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#28] +Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(cast(inv_quantity_on_hand#4 as bigint))#28 AS qoh#23] -(33) HashAggregate [codegen id : 16] +(32) HashAggregate [codegen id : 16] Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#23] Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#35, count#36] +Aggregate Attributes [2]: [sum#29, count#30] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#31, count#32] -(34) Exchange -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), true, [id=#37] +(33) Exchange +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#31, count#32] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), true, [id=#33] -(35) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#35, count#36] +(34) HashAggregate [codegen id : 17] +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#31, count#32] Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#38] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#39, avg(qoh#23)#38 AS qoh#40] +Aggregate Attributes [1]: [avg(qoh#23)#34] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#35, avg(qoh#23)#34 AS qoh#36] -(36) ReusedExchange [Reuses operator id: 27] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#41, count#42] +(35) ReusedExchange [Reuses operator id: 27] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#37, count#38] -(37) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#41, count#42] +(36) HashAggregate [codegen id : 25] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#37, count#38] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#43] -Results [3]: [i_product_name#15, i_brand#12, avg(cast(inv_quantity_on_hand#4 as bigint))#43 AS qoh#23] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#39] +Results [3]: [i_product_name#15, i_brand#12, avg(cast(inv_quantity_on_hand#4 as bigint))#39 AS qoh#23] -(38) HashAggregate [codegen id : 25] +(37) HashAggregate [codegen id : 25] Input [3]: [i_product_name#15, i_brand#12, qoh#23] Keys [2]: [i_product_name#15, i_brand#12] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#44, count#45] -Results [4]: [i_product_name#15, i_brand#12, sum#46, count#47] +Aggregate Attributes [2]: [sum#40, count#41] +Results [4]: [i_product_name#15, i_brand#12, sum#42, count#43] -(39) Exchange -Input [4]: [i_product_name#15, i_brand#12, sum#46, count#47] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), true, [id=#48] +(38) Exchange +Input [4]: [i_product_name#15, i_brand#12, sum#42, count#43] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), true, [id=#44] -(40) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#15, i_brand#12, sum#46, count#47] +(39) HashAggregate [codegen id : 26] +Input [4]: [i_product_name#15, i_brand#12, sum#42, count#43] Keys [2]: [i_product_name#15, i_brand#12] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#49] -Results [5]: [i_product_name#15, i_brand#12, null AS i_class#50, null AS i_category#51, avg(qoh#23)#49 AS qoh#52] +Aggregate Attributes [1]: [avg(qoh#23)#45] +Results [5]: [i_product_name#15, i_brand#12, null AS i_class#46, null AS i_category#47, avg(qoh#23)#45 AS qoh#48] -(41) ReusedExchange [Reuses operator id: 27] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#53, count#54] +(40) ReusedExchange [Reuses operator id: 27] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#49, count#50] -(42) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#53, count#54] +(41) HashAggregate [codegen id : 34] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#49, count#50] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#55] -Results [2]: [i_product_name#15, avg(cast(inv_quantity_on_hand#4 as bigint))#55 AS qoh#23] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#51] +Results [2]: [i_product_name#15, avg(cast(inv_quantity_on_hand#4 as bigint))#51 AS qoh#23] -(43) HashAggregate [codegen id : 34] +(42) HashAggregate [codegen id : 34] Input [2]: [i_product_name#15, qoh#23] Keys [1]: [i_product_name#15] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#56, count#57] -Results [3]: [i_product_name#15, sum#58, count#59] +Aggregate Attributes [2]: [sum#52, count#53] +Results [3]: [i_product_name#15, sum#54, count#55] -(44) Exchange -Input [3]: [i_product_name#15, sum#58, count#59] -Arguments: hashpartitioning(i_product_name#15, 5), true, [id=#60] +(43) Exchange +Input [3]: [i_product_name#15, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#15, 5), true, [id=#56] -(45) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#15, sum#58, count#59] +(44) HashAggregate [codegen id : 35] +Input [3]: [i_product_name#15, sum#54, count#55] Keys [1]: [i_product_name#15] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#61] -Results [5]: [i_product_name#15, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#23)#61 AS qoh#65] +Aggregate Attributes [1]: [avg(qoh#23)#57] +Results [5]: [i_product_name#15, null AS i_brand#58, null AS i_class#59, null AS i_category#60, avg(qoh#23)#57 AS qoh#61] -(46) ReusedExchange [Reuses operator id: 27] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] +(45) ReusedExchange [Reuses operator id: 27] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#62, count#63] -(47) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] +(46) HashAggregate [codegen id : 43] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#62, count#63] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#68] -Results [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#68 AS qoh#23] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#64] +Results [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#64 AS qoh#23] -(48) HashAggregate [codegen id : 43] +(47) HashAggregate [codegen id : 43] Input [1]: [qoh#23] Keys: [] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] +Aggregate Attributes [2]: [sum#65, count#66] +Results [2]: [sum#67, count#68] -(49) Exchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, true, [id=#73] +(48) Exchange +Input [2]: [sum#67, count#68] +Arguments: SinglePartition, true, [id=#69] -(50) HashAggregate [codegen id : 44] -Input [2]: [sum#71, count#72] +(49) HashAggregate [codegen id : 44] +Input [2]: [sum#67, count#68] Keys: [] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#74] -Results [5]: [null AS i_product_name#75, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#23)#74 AS qoh#79] +Aggregate Attributes [1]: [avg(qoh#23)#70] +Results [5]: [null AS i_product_name#71, null AS i_brand#72, null AS i_class#73, null AS i_category#74, avg(qoh#23)#70 AS qoh#75] -(51) Union +(50) Union -(52) TakeOrderedAndProject -Input [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#29] -Arguments: 100, [qoh#29 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#29] +(51) TakeOrderedAndProject +Input [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#25] +Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt index c2fe3189b2d10..afd4c2f2efc48 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt @@ -1,54 +1,53 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (8) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (7) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - SortMergeJoin [inv_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [inv_item_sk] - InputAdapter - Exchange [inv_item_sk] #2 - WholeStageCodegen (3) - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category] #1 + WholeStageCodegen (7) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + SortMergeJoin [inv_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [inv_item_sk] + InputAdapter + Exchange [inv_item_sk] #2 + WholeStageCodegen (3) + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (5) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + Scan parquet default.warehouse [w_warehouse_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (17) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 2a1ca82e1f263..9d5b70ae0d893 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -1,53 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- Union (48) - :- * HashAggregate (27) - : +- * HashAggregate (26) - : +- * HashAggregate (25) - : +- Exchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.inventory (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.item (11) - : +- BroadcastExchange (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet default.warehouse (17) - :- * HashAggregate (32) - : +- Exchange (31) - : +- * HashAggregate (30) - : +- * HashAggregate (29) - : +- ReusedExchange (28) - :- * HashAggregate (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- ReusedExchange (33) - :- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * HashAggregate (39) - : +- ReusedExchange (38) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- ReusedExchange (43) +TakeOrderedAndProject (48) ++- Union (47) + :- * HashAggregate (26) + : +- * HashAggregate (25) + : +- Exchange (24) + : +- * HashAggregate (23) + : +- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.inventory (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.item (11) + : +- BroadcastExchange (20) + : +- * Filter (19) + : +- * ColumnarToRow (18) + : +- Scan parquet default.warehouse (17) + :- * HashAggregate (31) + : +- Exchange (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- ReusedExchange (27) + :- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * HashAggregate (33) + : +- ReusedExchange (32) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- ReusedExchange (37) + +- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * HashAggregate (43) + +- ReusedExchange (42) (1) Scan parquet default.inventory @@ -170,132 +169,125 @@ Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(cast( (26) HashAggregate [codegen id : 5] Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#22] Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] -Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#25, count#26] - -(27) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#25, count#26] -Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#27] -Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#22)#27 AS qoh#28] +Aggregate Attributes [1]: [avg(qoh#22)#23] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#22)#23 AS qoh#24] -(28) ReusedExchange [Reuses operator id: 24] -Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#29, count#30] +(27) ReusedExchange [Reuses operator id: 24] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#25, count#26] -(29) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#29, count#30] +(28) HashAggregate [codegen id : 10] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#25, count#26] Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#31] -Results [4]: [i_product_name#12, i_brand#9, i_class#10, avg(cast(inv_quantity_on_hand#4 as bigint))#31 AS qoh#22] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#27] +Results [4]: [i_product_name#12, i_brand#9, i_class#10, avg(cast(inv_quantity_on_hand#4 as bigint))#27 AS qoh#22] -(30) HashAggregate [codegen id : 10] +(29) HashAggregate [codegen id : 10] Input [4]: [i_product_name#12, i_brand#9, i_class#10, qoh#22] Keys [3]: [i_product_name#12, i_brand#9, i_class#10] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [5]: [i_product_name#12, i_brand#9, i_class#10, sum#34, count#35] +Aggregate Attributes [2]: [sum#28, count#29] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, sum#30, count#31] -(31) Exchange -Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#34, count#35] -Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, 5), true, [id=#36] +(30) Exchange +Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#30, count#31] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, 5), true, [id=#32] -(32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#34, count#35] +(31) HashAggregate [codegen id : 11] +Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#30, count#31] Keys [3]: [i_product_name#12, i_brand#9, i_class#10] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#37] -Results [5]: [i_product_name#12, i_brand#9, i_class#10, null AS i_category#38, avg(qoh#22)#37 AS qoh#39] +Aggregate Attributes [1]: [avg(qoh#22)#33] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, null AS i_category#34, avg(qoh#22)#33 AS qoh#35] -(33) ReusedExchange [Reuses operator id: 24] -Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#40, count#41] +(32) ReusedExchange [Reuses operator id: 24] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#36, count#37] -(34) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#40, count#41] +(33) HashAggregate [codegen id : 16] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#36, count#37] Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#42] -Results [3]: [i_product_name#12, i_brand#9, avg(cast(inv_quantity_on_hand#4 as bigint))#42 AS qoh#22] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#38] +Results [3]: [i_product_name#12, i_brand#9, avg(cast(inv_quantity_on_hand#4 as bigint))#38 AS qoh#22] -(35) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 16] Input [3]: [i_product_name#12, i_brand#9, qoh#22] Keys [2]: [i_product_name#12, i_brand#9] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#43, count#44] -Results [4]: [i_product_name#12, i_brand#9, sum#45, count#46] +Aggregate Attributes [2]: [sum#39, count#40] +Results [4]: [i_product_name#12, i_brand#9, sum#41, count#42] -(36) Exchange -Input [4]: [i_product_name#12, i_brand#9, sum#45, count#46] -Arguments: hashpartitioning(i_product_name#12, i_brand#9, 5), true, [id=#47] +(35) Exchange +Input [4]: [i_product_name#12, i_brand#9, sum#41, count#42] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, 5), true, [id=#43] -(37) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#12, i_brand#9, sum#45, count#46] +(36) HashAggregate [codegen id : 17] +Input [4]: [i_product_name#12, i_brand#9, sum#41, count#42] Keys [2]: [i_product_name#12, i_brand#9] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#48] -Results [5]: [i_product_name#12, i_brand#9, null AS i_class#49, null AS i_category#50, avg(qoh#22)#48 AS qoh#51] +Aggregate Attributes [1]: [avg(qoh#22)#44] +Results [5]: [i_product_name#12, i_brand#9, null AS i_class#45, null AS i_category#46, avg(qoh#22)#44 AS qoh#47] -(38) ReusedExchange [Reuses operator id: 24] -Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#52, count#53] +(37) ReusedExchange [Reuses operator id: 24] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#48, count#49] -(39) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#52, count#53] +(38) HashAggregate [codegen id : 22] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#48, count#49] Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#54] -Results [2]: [i_product_name#12, avg(cast(inv_quantity_on_hand#4 as bigint))#54 AS qoh#22] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#50] +Results [2]: [i_product_name#12, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS qoh#22] -(40) HashAggregate [codegen id : 22] +(39) HashAggregate [codegen id : 22] Input [2]: [i_product_name#12, qoh#22] Keys [1]: [i_product_name#12] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#55, count#56] -Results [3]: [i_product_name#12, sum#57, count#58] +Aggregate Attributes [2]: [sum#51, count#52] +Results [3]: [i_product_name#12, sum#53, count#54] -(41) Exchange -Input [3]: [i_product_name#12, sum#57, count#58] -Arguments: hashpartitioning(i_product_name#12, 5), true, [id=#59] +(40) Exchange +Input [3]: [i_product_name#12, sum#53, count#54] +Arguments: hashpartitioning(i_product_name#12, 5), true, [id=#55] -(42) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#12, sum#57, count#58] +(41) HashAggregate [codegen id : 23] +Input [3]: [i_product_name#12, sum#53, count#54] Keys [1]: [i_product_name#12] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#60] -Results [5]: [i_product_name#12, null AS i_brand#61, null AS i_class#62, null AS i_category#63, avg(qoh#22)#60 AS qoh#64] +Aggregate Attributes [1]: [avg(qoh#22)#56] +Results [5]: [i_product_name#12, null AS i_brand#57, null AS i_class#58, null AS i_category#59, avg(qoh#22)#56 AS qoh#60] -(43) ReusedExchange [Reuses operator id: 24] -Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#65, count#66] +(42) ReusedExchange [Reuses operator id: 24] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#61, count#62] -(44) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#65, count#66] +(43) HashAggregate [codegen id : 28] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#61, count#62] Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#67] -Results [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#67 AS qoh#22] +Aggregate Attributes [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#63] +Results [1]: [avg(cast(inv_quantity_on_hand#4 as bigint))#63 AS qoh#22] -(45) HashAggregate [codegen id : 28] +(44) HashAggregate [codegen id : 28] Input [1]: [qoh#22] Keys: [] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#68, count#69] -Results [2]: [sum#70, count#71] +Aggregate Attributes [2]: [sum#64, count#65] +Results [2]: [sum#66, count#67] -(46) Exchange -Input [2]: [sum#70, count#71] -Arguments: SinglePartition, true, [id=#72] +(45) Exchange +Input [2]: [sum#66, count#67] +Arguments: SinglePartition, true, [id=#68] -(47) HashAggregate [codegen id : 29] -Input [2]: [sum#70, count#71] +(46) HashAggregate [codegen id : 29] +Input [2]: [sum#66, count#67] Keys: [] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#73] -Results [5]: [null AS i_product_name#74, null AS i_brand#75, null AS i_class#76, null AS i_category#77, avg(qoh#22)#73 AS qoh#78] +Aggregate Attributes [1]: [avg(qoh#22)#69] +Results [5]: [null AS i_product_name#70, null AS i_brand#71, null AS i_class#72, null AS i_category#73, avg(qoh#22)#69 AS qoh#74] -(48) Union +(47) Union -(49) TakeOrderedAndProject -Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#28] -Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#28] +(48) TakeOrderedAndProject +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] +Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 616bfc89c0023..da22d620755fa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,45 +1,44 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] - InputAdapter - Exchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Filter [w_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk] WholeStageCodegen (11) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index 6243d3234d187..72addf5120736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -1,84 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (80) -+- * Filter (79) - +- * HashAggregate (78) - +- * HashAggregate (77) - +- * Project (76) - +- * SortMergeJoin Inner (75) - :- * Filter (69) - : +- Window (68) - : +- * Sort (67) - : +- Exchange (66) - : +- * Project (65) - : +- * Filter (64) - : +- SortMergeJoin FullOuter (63) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * HashAggregate (31) - : : +- * HashAggregate (30) - : : +- * Project (29) - : : +- * SortMergeJoin Inner (28) - : : :- * Sort (20) - : : : +- Exchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- Window (16) - : : : +- * Sort (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- Exchange (12) - : : : +- * HashAggregate (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * Filter (24) - : : +- Window (23) - : : +- * Sort (22) - : : +- ReusedExchange (21) - : +- * Sort (62) - : +- Exchange (61) - : +- * HashAggregate (60) - : +- * HashAggregate (59) - : +- * Project (58) - : +- * SortMergeJoin Inner (57) - : :- * Sort (49) - : : +- Exchange (48) - : : +- * Project (47) - : : +- * Filter (46) - : : +- Window (45) - : : +- * Sort (44) - : : +- Exchange (43) - : : +- * HashAggregate (42) - : : +- Exchange (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.store_sales (34) - : : +- ReusedExchange (37) - : +- * Sort (56) - : +- Exchange (55) - : +- * Project (54) - : +- * Filter (53) - : +- Window (52) - : +- * Sort (51) - : +- ReusedExchange (50) - +- * Project (74) - +- * Filter (73) - +- Window (72) - +- * Sort (71) - +- ReusedExchange (70) +TakeOrderedAndProject (77) ++- * Filter (76) + +- * HashAggregate (75) + +- * Project (74) + +- * SortMergeJoin Inner (73) + :- * Filter (67) + : +- Window (66) + : +- * Sort (65) + : +- Exchange (64) + : +- * Project (63) + : +- * Filter (62) + : +- SortMergeJoin FullOuter (61) + : :- * Sort (32) + : : +- Exchange (31) + : : +- * HashAggregate (30) + : : +- * Project (29) + : : +- * SortMergeJoin Inner (28) + : : :- * Sort (20) + : : : +- Exchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- Window (16) + : : : +- * Sort (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- Exchange (12) + : : : +- * HashAggregate (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.web_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (27) + : : +- Exchange (26) + : : +- * Project (25) + : : +- * Filter (24) + : : +- Window (23) + : : +- * Sort (22) + : : +- ReusedExchange (21) + : +- * Sort (60) + : +- Exchange (59) + : +- * HashAggregate (58) + : +- * Project (57) + : +- * SortMergeJoin Inner (56) + : :- * Sort (48) + : : +- Exchange (47) + : : +- * Project (46) + : : +- * Filter (45) + : : +- Window (44) + : : +- * Sort (43) + : : +- Exchange (42) + : : +- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (35) + : : : +- * ColumnarToRow (34) + : : : +- Scan parquet default.store_sales (33) + : : +- ReusedExchange (36) + : +- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * Filter (52) + : +- Window (51) + : +- * Sort (50) + : +- ReusedExchange (49) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- ReusedExchange (68) (1) Scan parquet default.web_sales @@ -211,231 +208,210 @@ Input [7]: [item_sk#12, d_date#5, sumws#13, rk#15, item_sk#17, sumws#19, rk#20] (30) HashAggregate [codegen id : 13] Input [4]: [item_sk#12, d_date#5, sumws#13, sumws#19] Keys [3]: [item_sk#12, d_date#5, sumws#13] -Functions [1]: [partial_sum(sumws#19)] -Aggregate Attributes [2]: [sum#22, isEmpty#23] -Results [5]: [item_sk#12, d_date#5, sumws#13, sum#24, isEmpty#25] - -(31) HashAggregate [codegen id : 13] -Input [5]: [item_sk#12, d_date#5, sumws#13, sum#24, isEmpty#25] -Keys [3]: [item_sk#12, d_date#5, sumws#13] Functions [1]: [sum(sumws#19)] -Aggregate Attributes [1]: [sum(sumws#19)#26] -Results [3]: [item_sk#12, d_date#5, sum(sumws#19)#26 AS cume_sales#27] +Aggregate Attributes [1]: [sum(sumws#19)#22] +Results [3]: [item_sk#12, d_date#5, sum(sumws#19)#22 AS cume_sales#23] -(32) Exchange -Input [3]: [item_sk#12, d_date#5, cume_sales#27] -Arguments: hashpartitioning(item_sk#12, d_date#5, 5), true, [id=#28] +(31) Exchange +Input [3]: [item_sk#12, d_date#5, cume_sales#23] +Arguments: hashpartitioning(item_sk#12, d_date#5, 5), true, [id=#24] -(33) Sort [codegen id : 14] -Input [3]: [item_sk#12, d_date#5, cume_sales#27] +(32) Sort [codegen id : 14] +Input [3]: [item_sk#12, d_date#5, cume_sales#23] Arguments: [item_sk#12 ASC NULLS FIRST, d_date#5 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#29, ss_item_sk#30, ss_sales_price#31] +(33) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#25, ss_item_sk#26, ss_sales_price#27] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 16] -Input [3]: [ss_sold_date_sk#29, ss_item_sk#30, ss_sales_price#31] +(34) ColumnarToRow [codegen id : 16] +Input [3]: [ss_sold_date_sk#25, ss_item_sk#26, ss_sales_price#27] -(36) Filter [codegen id : 16] -Input [3]: [ss_sold_date_sk#29, ss_item_sk#30, ss_sales_price#31] -Condition : (isnotnull(ss_item_sk#30) AND isnotnull(ss_sold_date_sk#29)) +(35) Filter [codegen id : 16] +Input [3]: [ss_sold_date_sk#25, ss_item_sk#26, ss_sales_price#27] +Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_date_sk#25)) -(37) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#32, d_date#33] +(36) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#28, d_date#29] -(38) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#32] +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None -(39) Project [codegen id : 16] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#33] -Input [5]: [ss_sold_date_sk#29, ss_item_sk#30, ss_sales_price#31, d_date_sk#32, d_date#33] - -(40) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#33] -Keys [2]: [ss_item_sk#30, d_date#33] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#34] -Results [3]: [ss_item_sk#30, d_date#33, sum#35] - -(41) Exchange -Input [3]: [ss_item_sk#30, d_date#33, sum#35] -Arguments: hashpartitioning(ss_item_sk#30, d_date#33, 5), true, [id=#36] - -(42) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#30, d_date#33, sum#35] -Keys [2]: [ss_item_sk#30, d_date#33] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#37] -Results [4]: [ss_item_sk#30 AS item_sk#38, d_date#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#37,17,2) AS sumss#39, ss_item_sk#30] - -(43) Exchange -Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), true, [id=#40] - -(44) Sort [codegen id : 18] -Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST], false, 0 - -(45) Window -Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#41], [ss_item_sk#30], [d_date#33 ASC NULLS FIRST] - -(46) Filter [codegen id : 19] -Input [5]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#30, rk#41] -Condition : isnotnull(rk#41) - -(47) Project [codegen id : 19] -Output [4]: [item_sk#38, d_date#33, sumss#39, rk#41] -Input [5]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#30, rk#41] - -(48) Exchange -Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41] -Arguments: hashpartitioning(item_sk#38, 5), true, [id=#42] - -(49) Sort [codegen id : 20] -Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41] -Arguments: [item_sk#38 ASC NULLS FIRST], false, 0 - -(50) ReusedExchange [Reuses operator id: 43] -Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] - -(51) Sort [codegen id : 24] -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 - -(52) Window -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#30], [d_date#44 ASC NULLS FIRST] - -(53) Filter [codegen id : 25] -Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] -Condition : isnotnull(rk#46) - -(54) Project [codegen id : 25] -Output [3]: [item_sk#43, sumss#45, rk#46] -Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] - -(55) Exchange -Input [3]: [item_sk#43, sumss#45, rk#46] -Arguments: hashpartitioning(item_sk#43, 5), true, [id=#47] - -(56) Sort [codegen id : 26] -Input [3]: [item_sk#43, sumss#45, rk#46] -Arguments: [item_sk#43 ASC NULLS FIRST], false, 0 - -(57) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#38] -Right keys [1]: [item_sk#43] -Join condition: (rk#41 >= rk#46) - -(58) Project [codegen id : 27] -Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#45] -Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#43, sumss#45, rk#46] - -(59) HashAggregate [codegen id : 27] -Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#45] -Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [partial_sum(sumss#45)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [5]: [item_sk#38, d_date#33, sumss#39, sum#50, isEmpty#51] - -(60) HashAggregate [codegen id : 27] -Input [5]: [item_sk#38, d_date#33, sumss#39, sum#50, isEmpty#51] -Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [sum(sumss#45)] -Aggregate Attributes [1]: [sum(sumss#45)#52] -Results [3]: [item_sk#38, d_date#33, sum(sumss#45)#52 AS cume_sales#53] - -(61) Exchange -Input [3]: [item_sk#38, d_date#33, cume_sales#53] -Arguments: hashpartitioning(item_sk#38, d_date#33, 5), true, [id=#54] - -(62) Sort [codegen id : 28] -Input [3]: [item_sk#38, d_date#33, cume_sales#53] -Arguments: [item_sk#38 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST], false, 0 - -(63) SortMergeJoin +(38) Project [codegen id : 16] +Output [3]: [ss_item_sk#26, ss_sales_price#27, d_date#29] +Input [5]: [ss_sold_date_sk#25, ss_item_sk#26, ss_sales_price#27, d_date_sk#28, d_date#29] + +(39) HashAggregate [codegen id : 16] +Input [3]: [ss_item_sk#26, ss_sales_price#27, d_date#29] +Keys [2]: [ss_item_sk#26, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#27))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [ss_item_sk#26, d_date#29, sum#31] + +(40) Exchange +Input [3]: [ss_item_sk#26, d_date#29, sum#31] +Arguments: hashpartitioning(ss_item_sk#26, d_date#29, 5), true, [id=#32] + +(41) HashAggregate [codegen id : 17] +Input [3]: [ss_item_sk#26, d_date#29, sum#31] +Keys [2]: [ss_item_sk#26, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#27))#33] +Results [4]: [ss_item_sk#26 AS item_sk#34, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#27))#33,17,2) AS sumss#35, ss_item_sk#26] + +(42) Exchange +Input [4]: [item_sk#34, d_date#29, sumss#35, ss_item_sk#26] +Arguments: hashpartitioning(ss_item_sk#26, 5), true, [id=#36] + +(43) Sort [codegen id : 18] +Input [4]: [item_sk#34, d_date#29, sumss#35, ss_item_sk#26] +Arguments: [ss_item_sk#26 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 + +(44) Window +Input [4]: [item_sk#34, d_date#29, sumss#35, ss_item_sk#26] +Arguments: [row_number() windowspecdefinition(ss_item_sk#26, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#26], [d_date#29 ASC NULLS FIRST] + +(45) Filter [codegen id : 19] +Input [5]: [item_sk#34, d_date#29, sumss#35, ss_item_sk#26, rk#37] +Condition : isnotnull(rk#37) + +(46) Project [codegen id : 19] +Output [4]: [item_sk#34, d_date#29, sumss#35, rk#37] +Input [5]: [item_sk#34, d_date#29, sumss#35, ss_item_sk#26, rk#37] + +(47) Exchange +Input [4]: [item_sk#34, d_date#29, sumss#35, rk#37] +Arguments: hashpartitioning(item_sk#34, 5), true, [id=#38] + +(48) Sort [codegen id : 20] +Input [4]: [item_sk#34, d_date#29, sumss#35, rk#37] +Arguments: [item_sk#34 ASC NULLS FIRST], false, 0 + +(49) ReusedExchange [Reuses operator id: 42] +Output [4]: [item_sk#39, d_date#40, sumss#41, ss_item_sk#26] + +(50) Sort [codegen id : 24] +Input [4]: [item_sk#39, d_date#40, sumss#41, ss_item_sk#26] +Arguments: [ss_item_sk#26 ASC NULLS FIRST, d_date#40 ASC NULLS FIRST], false, 0 + +(51) Window +Input [4]: [item_sk#39, d_date#40, sumss#41, ss_item_sk#26] +Arguments: [row_number() windowspecdefinition(ss_item_sk#26, d_date#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#26], [d_date#40 ASC NULLS FIRST] + +(52) Filter [codegen id : 25] +Input [5]: [item_sk#39, d_date#40, sumss#41, ss_item_sk#26, rk#42] +Condition : isnotnull(rk#42) + +(53) Project [codegen id : 25] +Output [3]: [item_sk#39, sumss#41, rk#42] +Input [5]: [item_sk#39, d_date#40, sumss#41, ss_item_sk#26, rk#42] + +(54) Exchange +Input [3]: [item_sk#39, sumss#41, rk#42] +Arguments: hashpartitioning(item_sk#39, 5), true, [id=#43] + +(55) Sort [codegen id : 26] +Input [3]: [item_sk#39, sumss#41, rk#42] +Arguments: [item_sk#39 ASC NULLS FIRST], false, 0 + +(56) SortMergeJoin [codegen id : 27] +Left keys [1]: [item_sk#34] +Right keys [1]: [item_sk#39] +Join condition: (rk#37 >= rk#42) + +(57) Project [codegen id : 27] +Output [4]: [item_sk#34, d_date#29, sumss#35, sumss#41] +Input [7]: [item_sk#34, d_date#29, sumss#35, rk#37, item_sk#39, sumss#41, rk#42] + +(58) HashAggregate [codegen id : 27] +Input [4]: [item_sk#34, d_date#29, sumss#35, sumss#41] +Keys [3]: [item_sk#34, d_date#29, sumss#35] +Functions [1]: [sum(sumss#41)] +Aggregate Attributes [1]: [sum(sumss#41)#44] +Results [3]: [item_sk#34, d_date#29, sum(sumss#41)#44 AS cume_sales#45] + +(59) Exchange +Input [3]: [item_sk#34, d_date#29, cume_sales#45] +Arguments: hashpartitioning(item_sk#34, d_date#29, 5), true, [id=#46] + +(60) Sort [codegen id : 28] +Input [3]: [item_sk#34, d_date#29, cume_sales#45] +Arguments: [item_sk#34 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 + +(61) SortMergeJoin Left keys [2]: [item_sk#12, d_date#5] -Right keys [2]: [item_sk#38, d_date#33] +Right keys [2]: [item_sk#34, d_date#29] Join condition: None -(64) Filter [codegen id : 29] -Input [6]: [item_sk#12, d_date#5, cume_sales#27, item_sk#38, d_date#33, cume_sales#53] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#12) THEN item_sk#12 ELSE item_sk#38 END) - -(65) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#12) THEN item_sk#12 ELSE item_sk#38 END AS item_sk#55, CASE WHEN isnotnull(d_date#5) THEN d_date#5 ELSE d_date#33 END AS d_date#56, cume_sales#27 AS web_sales#57, cume_sales#53 AS store_sales#58] -Input [6]: [item_sk#12, d_date#5, cume_sales#27, item_sk#38, d_date#33, cume_sales#53] - -(66) Exchange -Input [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] -Arguments: hashpartitioning(item_sk#55, 5), true, [id=#59] - -(67) Sort [codegen id : 30] -Input [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] -Arguments: [item_sk#55 ASC NULLS FIRST, d_date#56 ASC NULLS FIRST], false, 0 - -(68) Window -Input [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] -Arguments: [row_number() windowspecdefinition(item_sk#55, d_date#56 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#60], [item_sk#55], [d_date#56 ASC NULLS FIRST] - -(69) Filter [codegen id : 31] -Input [5]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, rk#60] -Condition : isnotnull(rk#60) - -(70) ReusedExchange [Reuses operator id: 66] -Output [4]: [item_sk#61, d_date#62, web_sales#63, store_sales#64] - -(71) Sort [codegen id : 61] -Input [4]: [item_sk#61, d_date#62, web_sales#63, store_sales#64] -Arguments: [item_sk#61 ASC NULLS FIRST, d_date#62 ASC NULLS FIRST], false, 0 - -(72) Window -Input [4]: [item_sk#61, d_date#62, web_sales#63, store_sales#64] -Arguments: [row_number() windowspecdefinition(item_sk#61, d_date#62 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#65], [item_sk#61], [d_date#62 ASC NULLS FIRST] - -(73) Filter [codegen id : 62] -Input [5]: [item_sk#61, d_date#62, web_sales#63, store_sales#64, rk#65] -Condition : isnotnull(rk#65) - -(74) Project [codegen id : 62] -Output [4]: [item_sk#61, web_sales#63, store_sales#64, rk#65] -Input [5]: [item_sk#61, d_date#62, web_sales#63, store_sales#64, rk#65] - -(75) SortMergeJoin [codegen id : 63] -Left keys [1]: [item_sk#55] -Right keys [1]: [item_sk#61] -Join condition: (rk#60 >= rk#65) - -(76) Project [codegen id : 63] -Output [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_sales#63, store_sales#64] -Input [9]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, rk#60, item_sk#61, web_sales#63, store_sales#64, rk#65] - -(77) HashAggregate [codegen id : 63] -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_sales#63, store_sales#64] -Keys [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] -Functions [2]: [partial_max(web_sales#63), partial_max(store_sales#64)] -Aggregate Attributes [2]: [max#66, max#67] -Results [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max#68, max#69] - -(78) HashAggregate [codegen id : 63] -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max#68, max#69] -Keys [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] -Functions [2]: [max(web_sales#63), max(store_sales#64)] -Aggregate Attributes [2]: [max(web_sales#63)#70, max(store_sales#64)#71] -Results [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max(web_sales#63)#70 AS web_cumulative#72, max(store_sales#64)#71 AS store_cumulative#73] - -(79) Filter [codegen id : 63] -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#72, store_cumulative#73] -Condition : ((isnotnull(web_cumulative#72) AND isnotnull(store_cumulative#73)) AND (web_cumulative#72 > store_cumulative#73)) - -(80) TakeOrderedAndProject -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#72, store_cumulative#73] -Arguments: 100, [item_sk#55 ASC NULLS FIRST, d_date#56 ASC NULLS FIRST], [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#72, store_cumulative#73] +(62) Filter [codegen id : 29] +Input [6]: [item_sk#12, d_date#5, cume_sales#23, item_sk#34, d_date#29, cume_sales#45] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#12) THEN item_sk#12 ELSE item_sk#34 END) + +(63) Project [codegen id : 29] +Output [4]: [CASE WHEN isnotnull(item_sk#12) THEN item_sk#12 ELSE item_sk#34 END AS item_sk#47, CASE WHEN isnotnull(d_date#5) THEN d_date#5 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#45 AS store_sales#50] +Input [6]: [item_sk#12, d_date#5, cume_sales#23, item_sk#34, d_date#29, cume_sales#45] + +(64) Exchange +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: hashpartitioning(item_sk#47, 5), true, [id=#51] + +(65) Sort [codegen id : 30] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 + +(66) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] + +(67) Filter [codegen id : 31] +Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] +Condition : isnotnull(rk#52) + +(68) ReusedExchange [Reuses operator id: 64] +Output [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] + +(69) Sort [codegen id : 61] +Input [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] +Arguments: [item_sk#53 ASC NULLS FIRST, d_date#54 ASC NULLS FIRST], false, 0 + +(70) Window +Input [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] +Arguments: [row_number() windowspecdefinition(item_sk#53, d_date#54 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#57], [item_sk#53], [d_date#54 ASC NULLS FIRST] + +(71) Filter [codegen id : 62] +Input [5]: [item_sk#53, d_date#54, web_sales#55, store_sales#56, rk#57] +Condition : isnotnull(rk#57) + +(72) Project [codegen id : 62] +Output [4]: [item_sk#53, web_sales#55, store_sales#56, rk#57] +Input [5]: [item_sk#53, d_date#54, web_sales#55, store_sales#56, rk#57] + +(73) SortMergeJoin [codegen id : 63] +Left keys [1]: [item_sk#47] +Right keys [1]: [item_sk#53] +Join condition: (rk#52 >= rk#57) + +(74) Project [codegen id : 63] +Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#55, store_sales#56] +Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52, item_sk#53, web_sales#55, store_sales#56, rk#57] + +(75) HashAggregate [codegen id : 63] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#55, store_sales#56] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [max(web_sales#55), max(store_sales#56)] +Aggregate Attributes [2]: [max(web_sales#55)#58, max(store_sales#56)#59] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#55)#58 AS web_cumulative#60, max(store_sales#56)#59 AS store_cumulative#61] + +(76) Filter [codegen id : 63] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#60, store_cumulative#61] +Condition : ((isnotnull(web_cumulative#60) AND isnotnull(store_cumulative#61)) AND (web_cumulative#60 > store_cumulative#61)) + +(77) TakeOrderedAndProject +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#60, store_cumulative#61] +Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#60, store_cumulative#61] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt index 27e0d254e0d7f..908d3127a534d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt @@ -1,139 +1,136 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] WholeStageCodegen (63) Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - SortMergeJoin [item_sk,item_sk,rk,rk] - InputAdapter - WholeStageCodegen (31) + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + SortMergeJoin [item_sk,item_sk,rk,rk] + InputAdapter + WholeStageCodegen (31) + Filter [rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (30) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk] #1 + WholeStageCodegen (29) + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] + Filter [item_sk,item_sk] + InputAdapter + SortMergeJoin [item_sk,d_date,item_sk,d_date] + WholeStageCodegen (14) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #2 + WholeStageCodegen (13) + HashAggregate [item_sk,d_date,sumws,sumws] [sum(sumws),cume_sales,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + SortMergeJoin [item_sk,item_sk,rk,rk] + InputAdapter + WholeStageCodegen (6) + Sort [item_sk] + InputAdapter + Exchange [item_sk] #3 + WholeStageCodegen (5) + Project [item_sk,d_date,sumws,rk] + Filter [rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + Sort [ws_item_sk,d_date] + InputAdapter + Exchange [ws_item_sk] #4 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + Exchange [ws_item_sk,d_date] #5 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (12) + Sort [item_sk] + InputAdapter + Exchange [item_sk] #7 + WholeStageCodegen (11) + Project [item_sk,sumws,rk] + Filter [rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (10) + Sort [ws_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + WholeStageCodegen (28) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #8 + WholeStageCodegen (27) + HashAggregate [item_sk,d_date,sumss,sumss] [sum(sumss),cume_sales,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + SortMergeJoin [item_sk,item_sk,rk,rk] + InputAdapter + WholeStageCodegen (20) + Sort [item_sk] + InputAdapter + Exchange [item_sk] #9 + WholeStageCodegen (19) + Project [item_sk,d_date,sumss,rk] + Filter [rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (18) + Sort [ss_item_sk,d_date] + InputAdapter + Exchange [ss_item_sk] #10 + WholeStageCodegen (17) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + Exchange [ss_item_sk,d_date] #11 + WholeStageCodegen (16) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + WholeStageCodegen (26) + Sort [item_sk] + InputAdapter + Exchange [item_sk] #12 + WholeStageCodegen (25) + Project [item_sk,sumss,rk] + Filter [rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (24) + Sort [ss_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + InputAdapter + WholeStageCodegen (62) + Project [item_sk,web_sales,store_sales,rk] Filter [rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (30) + WholeStageCodegen (61) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (29) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - Filter [item_sk,item_sk] - InputAdapter - SortMergeJoin [item_sk,d_date,item_sk,d_date] - WholeStageCodegen (14) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (13) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - SortMergeJoin [item_sk,item_sk,rk,rk] - InputAdapter - WholeStageCodegen (6) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #3 - WholeStageCodegen (5) - Project [item_sk,d_date,sumws,rk] - Filter [rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (12) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #7 - WholeStageCodegen (11) - Project [item_sk,sumws,rk] - Filter [rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (10) - Sort [ws_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - WholeStageCodegen (28) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #8 - WholeStageCodegen (27) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - SortMergeJoin [item_sk,item_sk,rk,rk] - InputAdapter - WholeStageCodegen (20) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #9 - WholeStageCodegen (19) - Project [item_sk,d_date,sumss,rk] - Filter [rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (18) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #11 - WholeStageCodegen (16) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - WholeStageCodegen (26) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #12 - WholeStageCodegen (25) - Project [item_sk,sumss,rk] - Filter [rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (24) - Sort [ss_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 - InputAdapter - WholeStageCodegen (62) - Project [item_sk,web_sales,store_sales,rk] - Filter [rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (61) - Sort [item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 386b2da5528c7..92b1f54a3f1c9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,81 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- * Filter (65) - : +- Window (64) - : +- * Sort (63) - : +- Exchange (62) - : +- * Project (61) - : +- * Filter (60) - : +- SortMergeJoin FullOuter (59) - : :- * Sort (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (18) - : : : +- * Filter (17) - : : : +- Window (16) - : : : +- * Sort (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- Exchange (12) - : : : +- * HashAggregate (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (24) - : : +- * Project (23) - : : +- * Filter (22) - : : +- Window (21) - : : +- * Sort (20) - : : +- ReusedExchange (19) - : +- * Sort (58) - : +- Exchange (57) - : +- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (45) - : : +- * Filter (44) - : : +- Window (43) - : : +- * Sort (42) - : : +- Exchange (41) - : : +- * HashAggregate (40) - : : +- Exchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Filter (34) - : : : +- * ColumnarToRow (33) - : : : +- Scan parquet default.store_sales (32) - : : +- ReusedExchange (35) - : +- BroadcastExchange (51) - : +- * Project (50) - : +- * Filter (49) - : +- Window (48) - : +- * Sort (47) - : +- ReusedExchange (46) - +- BroadcastExchange (71) - +- * Project (70) - +- * Filter (69) - +- Window (68) - +- * Sort (67) - +- ReusedExchange (66) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- * Filter (65) + : +- Window (64) + : +- * Sort (63) + : +- Exchange (62) + : +- * Project (61) + : +- * Filter (60) + : +- SortMergeJoin FullOuter (59) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (18) + : : : +- * Filter (17) + : : : +- Window (16) + : : : +- * Sort (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- Exchange (12) + : : : +- * HashAggregate (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.web_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- Window (21) + : : +- * Sort (20) + : : +- ReusedExchange (19) + : +- * Sort (58) + : +- Exchange (57) + : +- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (45) + : : +- * Filter (44) + : : +- Window (43) + : : +- * Sort (42) + : : +- Exchange (41) + : : +- * HashAggregate (40) + : : +- Exchange (39) + : : +- * HashAggregate (38) + : : +- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet default.store_sales (32) + : : +- ReusedExchange (35) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- * Filter (49) + : +- Window (48) + : +- * Sort (47) + : +- ReusedExchange (46) + +- BroadcastExchange (71) + +- * Project (70) + +- * Filter (69) + +- Window (68) + +- * Sort (67) + +- ReusedExchange (66) (1) Scan parquet default.web_sales @@ -405,22 +404,15 @@ Input [9]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, rk#60, item_sk# (74) HashAggregate [codegen id : 54] Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_sales#63, store_sales#64] Keys [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] -Functions [2]: [partial_max(web_sales#63), partial_max(store_sales#64)] -Aggregate Attributes [2]: [max#67, max#68] -Results [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max#69, max#70] - -(75) HashAggregate [codegen id : 54] -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max#69, max#70] -Keys [4]: [item_sk#55, d_date#56, web_sales#57, store_sales#58] Functions [2]: [max(web_sales#63), max(store_sales#64)] -Aggregate Attributes [2]: [max(web_sales#63)#71, max(store_sales#64)#72] -Results [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max(web_sales#63)#71 AS web_cumulative#73, max(store_sales#64)#72 AS store_cumulative#74] +Aggregate Attributes [2]: [max(web_sales#63)#67, max(store_sales#64)#68] +Results [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, max(web_sales#63)#67 AS web_cumulative#69, max(store_sales#64)#68 AS store_cumulative#70] -(76) Filter [codegen id : 54] -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#73, store_cumulative#74] -Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) AND (web_cumulative#73 > store_cumulative#74)) +(75) Filter [codegen id : 54] +Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#69, store_cumulative#70] +Condition : ((isnotnull(web_cumulative#69) AND isnotnull(store_cumulative#70)) AND (web_cumulative#69 > store_cumulative#70)) -(77) TakeOrderedAndProject -Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#73, store_cumulative#74] -Arguments: 100, [item_sk#55 ASC NULLS FIRST, d_date#56 ASC NULLS FIRST], [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#73, store_cumulative#74] +(76) TakeOrderedAndProject +Input [6]: [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#69, store_cumulative#70] +Arguments: 100, [item_sk#55 ASC NULLS FIRST, d_date#56 ASC NULLS FIRST], [item_sk#55, d_date#56, web_sales#57, store_sales#58, web_cumulative#69, store_cumulative#70] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index c17a3085b030c..e1584c46e605f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,126 +1,125 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] WholeStageCodegen (54) Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Filter [rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (26) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (25) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - Filter [item_sk,item_sk] - InputAdapter - SortMergeJoin [item_sk,d_date,item_sk,d_date] - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - Filter [rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - Filter [rk] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Filter [rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (26) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk] #1 + WholeStageCodegen (25) + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] + Filter [item_sk,item_sk] + InputAdapter + SortMergeJoin [item_sk,d_date,item_sk,d_date] + WholeStageCodegen (12) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #2 + WholeStageCodegen (11) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + InputAdapter + Exchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + Filter [rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + Sort [ws_item_sk,d_date] InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - Sort [ws_item_sk,d_date] + Exchange [ws_item_sk] #4 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - WholeStageCodegen (24) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #8 - WholeStageCodegen (23) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - Filter [rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (16) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (15) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #11 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - Project [item_sk,sumss,rk] - Filter [rk] + Exchange [ws_item_sk,d_date] #5 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + Project [item_sk,sumws,rk] + Filter [rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (8) + Sort [ws_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + WholeStageCodegen (24) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #8 + WholeStageCodegen (23) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + InputAdapter + Exchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (22) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + Filter [rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (16) + Sort [ss_item_sk,d_date] InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (20) - Sort [ss_item_sk,d_date] + Exchange [ss_item_sk] #10 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (53) - Project [item_sk,web_sales,store_sales,rk] - Filter [rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (52) - Sort [item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + Exchange [ss_item_sk,d_date] #11 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (21) + Project [item_sk,sumss,rk] + Filter [rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (20) + Sort [ss_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (53) + Project [item_sk,web_sales,store_sales,rk] + Filter [rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (52) + Sort [item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index d34dcb4fe0c01..bc4c013150e78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.Uuid +import org.apache.spark.sql.catalyst.expressions.aggregate.Complete import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, OneRowRelation} import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -1635,16 +1636,16 @@ class DataFrameSuite extends QueryTest /** * Verifies that there is no Exchange between the Aggregations for `df` */ - private def verifyNonExchangingAgg(df: DataFrame) = { + private def verifyCompleteAgg(df: DataFrame) = { var atFirstAgg: Boolean = false - df.queryExecution.executedPlan.foreach { - case agg: HashAggregateExec => - atFirstAgg = !atFirstAgg - case _ => - if (atFirstAgg) { - fail("Should not have operators between the two aggregations") - } + val aggs = df.queryExecution.executedPlan.collect { + case agg: HashAggregateExec => agg + } + if (aggs.size != 1) { + fail("Should have only 1 physical aggregation operator") } + assert(aggs.head.aggregateExpressions.forall(_.mode == Complete), + "aggregation mode should be complete") } /** @@ -1674,11 +1675,11 @@ class DataFrameSuite extends QueryTest assert(df2.rdd.partitions.length == 10) checkAnswer(original.select(), df2.select()) - // Group by the column we are distributed by. This should generate a plan with no exchange - // between the aggregates + // Group by the column we are distributed by. This should generate a plan with + // single HaPhysical aggregate node val df3 = testData.repartition($"key").groupBy("key").count() - verifyNonExchangingAgg(df3) - verifyNonExchangingAgg(testData.repartition($"key", $"value") + verifyCompleteAgg(df3) + verifyCompleteAgg(testData.repartition($"key", $"value") .groupBy("key", "value").count()) // Grouping by just the first distributeBy expr, need to exchange. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CollapseAggregatesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CollapseAggregatesSuite.scala new file mode 100644 index 0000000000000..041b1b7fc2a8d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CollapseAggregatesSuite.scala @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import scala.reflect.ClassTag + +import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +abstract class CollapseAggregatesSuiteBase + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + spark.sparkContext.parallelize(1 to 100, 10).toDF("id") + .selectExpr("id as col1", "id % 10 as col2").createOrReplaceTempView("t1") + spark.sparkContext.parallelize(1 to 100, 10).toDF("id") + .selectExpr("id as col1", "id % 10 as col2").createOrReplaceTempView("t2") + } + + override def afterAll(): Unit = { + try spark.catalog.dropTempView("t1") catch { + case _: NoSuchTableException => + } + try spark.catalog.dropTempView("t2") catch { + case _: NoSuchTableException => + } + super.afterAll() + } + + private def assertAggregateExecCount[T: ClassTag](df: DataFrame, expected: Int) = { + withClue(df.queryExecution) { + val plan = df.queryExecution.executedPlan + val actual = collectWithSubqueries(plan) { case agg: T => agg }.size + assert(actual == expected) + } + } + + private def assertBaseAggregateExec[T: ClassTag]( + query: String, + enabled: Int, + disabled: Int): Unit = { + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withSQLConf(SQLConf.COLLAPSE_AGGREGATE_NODES_ENABLED.key -> "false") { + val df1 = sql(query) + assertAggregateExecCount[T](df1, disabled) + val result = df1.collect() + withSQLConf(SQLConf.COLLAPSE_AGGREGATE_NODES_ENABLED.key -> "true") { + val df2 = sql(query) + assertAggregateExecCount[T](df2, enabled) + checkAnswer(df2, result) + } + } + } + } + + private def assertHashAggregateExec(query: String, enabled: Int, disabled: Int): Unit = { + assertBaseAggregateExec[HashAggregateExec](query, enabled, disabled) + } + + private def assertObjectHashAggregateExec(query: String, enabled: Int, disabled: Int): Unit = { + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + assertBaseAggregateExec[ObjectHashAggregateExec](query, enabled, disabled) + } + } + + private def assertSortAggregateExec(query: String, enabled: Int, disabled: Int): Unit = { + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + assertBaseAggregateExec[SortAggregateExec](query, enabled, disabled) + } + } + + test("hash aggregate") { + val query = "select max(col1), col2 from t1 group by col2" + assertHashAggregateExec(query, 2, 2) + } + + test("Aggregate after Aggregate where aggregates cannot be merged (HashAggregateExec)") { + val query = "select max(col1), min(c1), col2 " + + "from (select count(*) as c1, col1, col2 from t1 group by col1, col2) temp " + + "group by col2" + assertHashAggregateExec(query, 4, 4) + } + + test("Aggregate after aggregate where aggregates can be merged (HashAggregateExec)") { + val query = "select sum(c2), col1, c1 " + + "from (select col1, count(*) as c1, max(col2) as c2 from t1 group by col1) temp " + + "group by col1, c1" + assertHashAggregateExec(query, 3, 4) + } + + test("Aggregate after Join when it cannot be merged (HashAggregateExec)") { + val query = "select sum(t1.col1), t1.col2 " + + "from t1, t2 where t1.col1 = t2.col1 " + + "group by t1.col2" + assertHashAggregateExec(query, 2, 2) + } + + test("Aggregate after Join when it can be merged (HashAggregateExec)") { + val query = "select sum(t2.col1), max(t2.col2), t1.col1, t1.col2 " + + "from t1, t2 where t1.col1 = t2.col1 " + + "group by t1.col1, t1.col2" + assertHashAggregateExec(query, 1, 2) + } + + Seq(true, false).foreach { useObjectHashAgg => + test(s"object/sort aggregate [useObjectHashAgg: $useObjectHashAgg]") { + val query = "select collect_list(col1), col2 from t1 group by col2" + if (useObjectHashAgg) { + assertObjectHashAggregateExec(query, 2, 2) + } else { + assertSortAggregateExec(query, 2, 2) + } + } + + test("Aggregate after Aggregate where aggregates cannot be merged " + + s"[useObjectHashAgg: $useObjectHashAgg]") { + val query = "select max(col1), collect_list(c1), col2 " + + "from (select collect_list(1) as c1, col1, col2 from t1 group by col1, col2) temp " + + "group by col2" + + if (useObjectHashAgg) { + assertObjectHashAggregateExec(query, 4, 4) + } else { + assertSortAggregateExec(query, 4, 4) + } + } + + test("Aggregate after aggregate where aggregates can be merged " + + s"[useObjectHashAgg: $useObjectHashAgg]") { + val query = "select collect_list(c2), col1, c1 " + + "from (select col1, count(*) as c1, collect_list(col2) as c2 from t1 group by col1) temp " + + "group by col1, c1" + if (useObjectHashAgg) { + assertObjectHashAggregateExec(query, 3, 4) + } else { + assertSortAggregateExec(query, 3, 4) + } + } + + test(s"Aggregate after Join when it cannot be merged " + + s"[useObjectHashAgg: $useObjectHashAgg]") { + val query = "select collect_list(t1.col1), t1.col2 " + + "from t1, t2 where t1.col1 = t2.col1 " + + "group by t1.col2" + if (useObjectHashAgg) { + assertObjectHashAggregateExec(query, 2, 2) + } else { + assertSortAggregateExec(query, 2, 2) + } + } + + test("Aggregate after Join when it can be merged " + + s"[useObjectHashAgg: $useObjectHashAgg]") { + val query = "select collect_list(t2.col1), max(t2.col2), t1.col1, t1.col2 " + + "from t1, t2 where t1.col1 = t2.col1 " + + "group by t1.col1, t1.col2" + if (useObjectHashAgg) { + assertObjectHashAggregateExec(query, 1, 2) + } else { + assertSortAggregateExec(query, 1, 2) + } + } + } +} + +class CollapseAggregatesSuite extends CollapseAggregatesSuiteBase + with DisableAdaptiveExecutionSuite + +class CollapseAggregatesSuiteAE extends CollapseAggregatesSuiteBase + with EnableAdaptiveExecutionSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 4872906dbfec3..f87f68a8030a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.{classTag, ClassTag} import scala.util.Random import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} +import org.apache.spark.sql.catalyst.expressions.aggregate.Complete import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite @@ -669,13 +669,11 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils assert(ranges.head.metrics("numOutputRows").value == 1000) val aggs = collectNodeWithinWholeStage[HashAggregateExec](plan) - assert(aggs.length == 2, "The query plan should have two and only two Aggregate") - val partialAgg = aggs.filter(_.aggregateExpressions.head.mode == Partial).head - // The partial aggregate should output 10 rows, because its input is 10 rows. - assert(partialAgg.metrics("numOutputRows").value == 10) - val finalAgg = aggs.filter(_.aggregateExpressions.head.mode == Final).head - // The final aggregate should only produce 1 row, because the upstream limit only needs 1 row. - assert(finalAgg.metrics("numOutputRows").value == 1) + assert(aggs.length == 1, "The query plan should have only one Aggregate") + val agg = aggs.head + assert(agg.aggregateExpressions.head.mode == Complete, + "the aggregation mode for aggregate should be complete") + assert(agg.metrics("numOutputRows").value == 1) val filters = collectNodeWithinWholeStage[FilterExec](plan) assert(filters.length == 1, "The query plan should have one and only one Filter") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index ed44dcd8d7a29..2d04da2c6f354 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -98,7 +98,8 @@ class HiveUDAFSuite extends QueryTest test("SPARK-24935: customized Hive UDAF with two aggregation buffers") { withTempView("v") { - spark.range(100).createTempView("v") + // Setting numPartitions > 1 explicitly so that we get two Physical aggregation nodes + spark.range(0, 100, 1, 10).createTempView("v") val df = sql("SELECT id % 2, mock2(id) FROM v GROUP BY id % 2") val aggs = collect(df.queryExecution.executedPlan) {