From 6448864db368c323ca9fdc8651fd2abe2fa56425 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Wed, 24 Nov 2021 15:25:47 -0800 Subject: [PATCH 1/5] Replace hash agg with sort agg if child is already sorted --- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../spark/sql/execution/QueryExecution.scala | 3 + .../execution/ReplaceHashWithSortAgg.scala | 139 ++++++++++++++++++ .../adaptive/AdaptiveSparkPlanExec.scala | 1 + .../aggregate/HashAggregateExec.scala | 9 ++ .../ReplaceHashWithSortAggSuite.scala | 119 +++++++++++++++ 6 files changed, 278 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala 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 c7535c3cebe51..6e8ad266e4d74 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 @@ -1504,6 +1504,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val REPLACE_HASH_WITH_SORT_AGG_ENABLED = buildConf("spark.sql.execution.replaceHashWithSortAgg") + .internal() + .doc("Whether to replace hash aggregate node with sort aggregate based on children's ordering") + .version("3.3.0") + .booleanConf + .createWithDefault(true) + val STATE_STORE_PROVIDER_CLASS = buildConf("spark.sql.streaming.stateStore.providerClass") .internal() 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 bb1b0ca3b645a..26c6904a896a5 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 @@ -423,6 +423,9 @@ object QueryExecution { PlanSubqueries(sparkSession), RemoveRedundantProjects, EnsureRequirements(), + // `ReplaceHashWithSortAgg` needs to be added after `EnsureRequirements` to guarantee the + // sort order of each node is checked to be valid. + ReplaceHashWithSortAgg, // `RemoveRedundantSorts` needs to be added after `EnsureRequirements` to guarantee the same // number of partitions when instantiating PartitioningCollection. RemoveRedundantSorts, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala new file mode 100644 index 0000000000000..c259795dc45e7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala @@ -0,0 +1,139 @@ +/* + * 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.SortOrder +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.HashAggregateExec +import org.apache.spark.sql.internal.SQLConf + +/** + * Replace [[HashAggregateExec]] with [[SortAggregateExec]] in the spark plan if: + * + * 1. The plan is a pair of partial and final [[HashAggregateExec]], and the child of partial + * aggregate satisfies the sort order of corresponding [[SortAggregateExec]]. + * or + * 2. The plan is a [[HashAggregateExec]], and the child satisfies the sort order of + * corresponding [[SortAggregateExec]]. + * + * Examples: + * 1. aggregate after join: + * + * HashAggregate(t1.i, SUM, final) + * | SortAggregate(t1.i, SUM, complete) + * HashAggregate(t1.i, SUM, partial) => | + * | SortMergeJoin(t1.i = t2.j) + * SortMergeJoin(t1.i = t2.j) + * + * 2. aggregate after sort: + * + * HashAggregate(t1.i, SUM, partial) SortAggregate(t1.i, SUM, partial) + * | => | + * Sort(t1.i) Sort(t1.i) + * + * [[HashAggregateExec]] can be replaced when its child satisfies the sort order of + * corresponding [[SortAggregateExec]]. [[SortAggregateExec]] is faster in the sense that + * it does not have hashing overhead of [[HashAggregateExec]]. + */ +object ReplaceHashWithSortAgg extends Rule[SparkPlan] { + def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED)) { + plan + } else { + replaceHashAgg(plan) + } + } + + /** + * Replace [[HashAggregateExec]] with [[SortAggregateExec]]. + */ + private def replaceHashAgg(plan: SparkPlan): SparkPlan = { + plan.transformDown { + case hashAgg: HashAggregateExec => + val sortAgg = hashAgg.toSortAggregate + hashAgg.child match { + case partialAgg: HashAggregateExec if isPartialAgg(partialAgg, hashAgg) => + if (SortOrder.orderingSatisfies( + partialAgg.child.outputOrdering, sortAgg.requiredChildOrdering.head)) { + sortAgg.copy( + aggregateExpressions = sortAgg.aggregateExpressions.map(_.copy(mode = Complete)), + child = partialAgg.child) + } else { + hashAgg + } + case other => + if (SortOrder.orderingSatisfies( + other.outputOrdering, sortAgg.requiredChildOrdering.head)) { + sortAgg + } else { + hashAgg + } + } + case other => other + } + } + + /** + * Check if `partialAgg` to be partial aggregate of `finalAgg`. + */ + private def isPartialAgg(partialAgg: HashAggregateExec, finalAgg: HashAggregateExec): Boolean = { + val partialGroupExprs = partialAgg.groupingExpressions + val finalGroupExprs = finalAgg.groupingExpressions + val partialAggExprs = partialAgg.aggregateExpressions + val finalAggExprs = finalAgg.aggregateExpressions + val partialAggAttrs = partialAggExprs.flatMap(_.aggregateFunction.aggBufferAttributes) + val finalAggAttrs = finalAggExprs.map(_.resultAttribute) + val partialResultExprs = partialGroupExprs ++ + partialAggExprs.flatMap(_.aggregateFunction.inputAggBufferAttributes) + + val groupExprsEqual = partialGroupExprs.length == finalGroupExprs.length && + partialGroupExprs.zip(finalGroupExprs).forall { + case (e1, e2) => e1.semanticEquals(e2) + } + val aggExprsEqual = partialAggExprs.length == finalAggExprs.length && + partialAggExprs.forall(_.mode == Partial) && finalAggExprs.forall(_.mode == Final) && + partialAggExprs.zip(finalAggExprs).forall { + case (e1, e2) => e1.aggregateFunction.semanticEquals(e2.aggregateFunction) + } + val isPartialAggAttrsValid = partialAggAttrs.length == partialAgg.aggregateAttributes.length && + partialAggAttrs.zip(partialAgg.aggregateAttributes).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + val isFinalAggAttrsValid = finalAggAttrs.length == finalAgg.aggregateAttributes.length && + finalAggAttrs.zip(finalAgg.aggregateAttributes).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + val isPartialResultExprsValid = + partialResultExprs.length == partialAgg.resultExpressions.length && + partialResultExprs.zip(partialAgg.resultExpressions).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + val isRequiredDistributionValid = + partialAgg.requiredChildDistributionExpressions.isEmpty && + finalAgg.requiredChildDistributionExpressions.exists { exprs => + exprs.length == finalGroupExprs.length && + exprs.zip(finalGroupExprs).forall { + case (e1, e2) => e1.semanticEquals(e2) + } + } + + groupExprsEqual && aggExprsEqual && isPartialAggAttrsValid && isFinalAggAttrsValid && + isPartialResultExprsValid && isRequiredDistributionValid + } +} 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 a4fe4529ed3d4..2b42804e784ed 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 @@ -116,6 +116,7 @@ case class AdaptiveSparkPlanExec( Seq( RemoveRedundantProjects, ensureRequirements, + ReplaceHashWithSortAgg, RemoveRedundantSorts, DisableUnnecessaryBucketedScan, OptimizeSkewedJoin(ensureRequirements) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 8545154028602..85e81cb12dca4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -1153,6 +1153,15 @@ case class HashAggregateExec( } } + /** + * The corresponding [[SortAggregateExec]] to get same result as this node. + */ + def toSortAggregate: SortAggregateExec = { + SortAggregateExec( + requiredChildDistributionExpressions, groupingExpressions, aggregateExpressions, + aggregateAttributes, initialInputBufferOffset, resultExpressions, child) + } + override protected def withNewChildInternal(newChild: SparkPlan): HashAggregateExec = copy(child = newChild) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala new file mode 100644 index 0000000000000..1128bff13af65 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala @@ -0,0 +1,119 @@ +/* + * 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.{DataFrame, QueryTest} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +abstract class ReplaceHashWithSortAggSuiteBase + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + + private def checkNumAggs(df: DataFrame, hashAggCount: Int, sortAggCount: Int): Unit = { + val plan = df.queryExecution.executedPlan + assert(collectWithSubqueries(plan) { case s: HashAggregateExec => s }.length == hashAggCount) + assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) + } + + private def checkAggs( + query: String, + enabledHashAggCount: Int, + enabledSortAggCount: Int, + disabledHashAggCount: Int, + disabledSortAggCount: Int): Unit = { + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "true") { + val df = sql(query) + checkNumAggs(df, enabledHashAggCount, enabledSortAggCount) + val result = df.collect() + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "false") { + val df = sql(query) + checkNumAggs(df, disabledHashAggCount, disabledSortAggCount) + checkAnswer(df, result) + } + } + } + + test("replace partial hash aggregate with sort aggregate") { + withTempView("t") { + spark.range(100).selectExpr("id as key").repartition(10).createOrReplaceTempView("t") + val query = + """ + |SELECT key, FIRST(key) + |FROM + |( + | SELECT key + | FROM t + | WHERE key > 10 + | SORT BY key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 1, 1, 2, 0) + } + } + + test("replace partial and final hash aggregate together with sort aggregate") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + val query = + """ + |SELECT key, COUNT(key) + |FROM + |( + | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 0, 1, 2, 0) + } + } + + test("do not replace hash aggregate if child does not have sort order") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + val query = + """ + |SELECT key, COUNT(key) + |FROM + |( + | SELECT /*+ BROADCAST(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } +} + +class ReplaceHashWithSortAggSuite extends ReplaceHashWithSortAggSuiteBase + with DisableAdaptiveExecutionSuite + +class ReplaceHashWithSortAggSuiteAE extends ReplaceHashWithSortAggSuiteBase + with EnableAdaptiveExecutionSuite From a683137de1b6b6ab247f94a0ebf7a41c46ee8c62 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Thu, 25 Nov 2021 19:30:45 -0800 Subject: [PATCH 2/5] Do not replace hash agg if grouping expression is empty --- .../spark/sql/execution/ReplaceHashWithSortAgg.scala | 2 +- .../sql/execution/ReplaceHashWithSortAggSuite.scala | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala index c259795dc45e7..77a0681ca9876 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala @@ -65,7 +65,7 @@ object ReplaceHashWithSortAgg extends Rule[SparkPlan] { */ private def replaceHashAgg(plan: SparkPlan): SparkPlan = { plan.transformDown { - case hashAgg: HashAggregateExec => + case hashAgg: HashAggregateExec if hashAgg.groupingExpressions.nonEmpty => val sortAgg = hashAgg.toSortAggregate hashAgg.child match { case partialAgg: HashAggregateExec if isPartialAgg(partialAgg, hashAgg) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala index 1128bff13af65..78765fdf4f757 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala @@ -110,6 +110,18 @@ abstract class ReplaceHashWithSortAggSuiteBase checkAggs(query, 2, 0, 2, 0) } } + + test("do not replace hash aggregate if there is no group-by column") { + withTempView("t1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + val query = + """ + |SELECT COUNT(key) + |FROM t1 + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } } class ReplaceHashWithSortAggSuite extends ReplaceHashWithSortAggSuiteBase From e8609fdecc05dc74c60b4621683fc48bbfed5eaf Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Fri, 26 Nov 2021 17:08:49 -0800 Subject: [PATCH 3/5] Fix unit test failure --- .../approved-plans-v1_4/q16.sf100/explain.txt | 10 +- .../q16.sf100/simplified.txt | 126 +-- .../approved-plans-v1_4/q16/explain.txt | 10 +- .../approved-plans-v1_4/q16/simplified.txt | 126 +-- .../q23a.sf100/explain.txt | 604 ++++++------ .../q23a.sf100/simplified.txt | 71 +- .../q23b.sf100/explain.txt | 856 ++++++++---------- .../q23b.sf100/simplified.txt | 121 +-- .../approved-plans-v1_4/q54.sf100/explain.txt | 346 ++++--- .../q54.sf100/simplified.txt | 249 ++--- .../approved-plans-v1_4/q94.sf100/explain.txt | 10 +- .../q94.sf100/simplified.txt | 126 +-- .../approved-plans-v1_4/q94/explain.txt | 10 +- .../approved-plans-v1_4/q94/simplified.txt | 126 +-- .../approved-plans-v1_4/q95.sf100/explain.txt | 10 +- .../q95.sf100/simplified.txt | 176 ++-- .../approved-plans-v1_4/q95/explain.txt | 10 +- .../approved-plans-v1_4/q95/simplified.txt | 176 ++-- .../approved-plans-v2_7/q64.sf100/explain.txt | 4 +- .../q64.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q64/explain.txt | 4 +- .../approved-plans-v2_7/q64/simplified.txt | 4 +- .../execution/metric/SQLMetricsSuite.scala | 3 +- 23 files changed, 1548 insertions(+), 1634 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index cd26a5c2495dd..008ebc4c7ea98 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * HashAggregate (41) + +- SortAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#23] -(41) HashAggregate [codegen id : 11] +(41) SortAggregate Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 12] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 12] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 13] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 7af14cd98449e..4c4cd8d785925 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -1,74 +1,76 @@ -WholeStageCodegen (12) +WholeStageCodegen (13) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] + InputAdapter + SortAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + WholeStageCodegen (11) + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + Project [cs_warehouse_sk,cs_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [cr_order_number] + InputAdapter + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + Project [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] + ColumnarToRow InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [cc_call_center_sk] + Filter [cc_county,cc_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.call_center [cc_call_center_sk,cc_county] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [cc_call_center_sk] - Filter [cc_county,cc_call_center_sk] + BroadcastExchange #7 + WholeStageCodegen (10) + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 210be116962bb..4279f75e0a84d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * HashAggregate (41) + +- SortAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#23] -(41) HashAggregate [codegen id : 11] +(41) SortAggregate Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 12] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 12] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 13] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 126e03fa31d76..d4624703912a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,74 +1,76 @@ -WholeStageCodegen (12) +WholeStageCodegen (13) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] + InputAdapter + SortAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + WholeStageCodegen (11) + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + Project [cs_warehouse_sk,cs_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [cr_order_number] + InputAdapter + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + Project [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [d_date_sk] - Filter [d_date,d_date_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #7 + WholeStageCodegen (10) + Project [cc_call_center_sk] + Filter [cc_county,cc_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [cc_call_center_sk] - Filter [cc_county,cc_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_county] + Scan parquet default.call_center [cc_call_center_sk,cc_county] 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 be706fee66776..ba2475adc2ae6 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,12 +1,12 @@ == Physical Plan == -* HashAggregate (90) -+- Exchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (48) - : : +- * SortMergeJoin LeftSemi (47) +* HashAggregate (86) ++- Exchange (85) + +- * HashAggregate (84) + +- Union (83) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (46) + : : +- * SortMergeJoin LeftSemi (45) : : :- * Sort (28) : : : +- Exchange (27) : : : +- * Project (26) @@ -35,60 +35,56 @@ : : : +- * Filter (15) : : : +- * ColumnarToRow (14) : : : +- Scan parquet default.item (13) - : : +- * Sort (46) - : : +- * Project (45) - : : +- * Filter (44) - : : +- * HashAggregate (43) - : : +- * HashAggregate (42) - : : +- * Project (41) - : : +- * SortMergeJoin Inner (40) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * Filter (31) - : : : +- * ColumnarToRow (30) - : : : +- Scan parquet default.store_sales (29) - : : +- * Sort (39) - : : +- Exchange (38) - : : +- * Filter (37) - : : +- * ColumnarToRow (36) - : : +- Scan parquet default.customer (35) - : +- ReusedExchange (49) - +- * Project (86) - +- * BroadcastHashJoin Inner BuildRight (85) - :- * Project (83) - : +- * SortMergeJoin LeftSemi (82) - : :- * Sort (70) - : : +- Exchange (69) - : : +- * Project (68) - : : +- * SortMergeJoin LeftSemi (67) - : : :- * Sort (55) - : : : +- Exchange (54) - : : : +- * ColumnarToRow (53) - : : : +- Scan parquet default.web_sales (52) - : : +- * Sort (66) - : : +- * Project (65) - : : +- * Filter (64) - : : +- * HashAggregate (63) - : : +- * HashAggregate (62) - : : +- * Project (61) - : : +- * SortMergeJoin Inner (60) - : : :- * Sort (57) - : : : +- ReusedExchange (56) - : : +- * Sort (59) - : : +- ReusedExchange (58) - : +- * Sort (81) - : +- * Project (80) - : +- * Filter (79) - : +- * HashAggregate (78) - : +- * HashAggregate (77) - : +- * Project (76) - : +- * SortMergeJoin Inner (75) - : :- * Sort (72) - : : +- ReusedExchange (71) - : +- * Sort (74) - : +- ReusedExchange (73) - +- ReusedExchange (84) + : : +- * Project (44) + : : +- * Filter (43) + : : +- SortAggregate (42) + : : +- * Project (41) + : : +- * SortMergeJoin Inner (40) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * Filter (31) + : : : +- * ColumnarToRow (30) + : : : +- Scan parquet default.store_sales (29) + : : +- * Sort (39) + : : +- Exchange (38) + : : +- * Filter (37) + : : +- * ColumnarToRow (36) + : : +- Scan parquet default.customer (35) + : +- ReusedExchange (47) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * SortMergeJoin LeftSemi (78) + : :- * Sort (68) + : : +- Exchange (67) + : : +- * Project (66) + : : +- * SortMergeJoin LeftSemi (65) + : : :- * Sort (53) + : : : +- Exchange (52) + : : : +- * ColumnarToRow (51) + : : : +- Scan parquet default.web_sales (50) + : : +- * Sort (64) + : : +- * Project (63) + : : +- * Filter (62) + : : +- * HashAggregate (61) + : : +- * HashAggregate (60) + : : +- * Project (59) + : : +- * SortMergeJoin Inner (58) + : : :- * Sort (55) + : : : +- ReusedExchange (54) + : : +- * Sort (57) + : : +- ReusedExchange (56) + : +- * Project (77) + : +- * Filter (76) + : +- SortAggregate (75) + : +- * Project (74) + : +- * SortMergeJoin Inner (73) + : :- * Sort (70) + : : +- ReusedExchange (69) + : +- * Sort (72) + : +- ReusedExchange (71) + +- ReusedExchange (80) (1) Scan parquet default.catalog_sales @@ -124,7 +120,7 @@ Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(8) ReusedExchange [Reuses operator id: 100] +(8) ReusedExchange [Reuses operator id: 96] Output [2]: [d_date_sk#11, d_date#12] (9) BroadcastHashJoin [codegen id : 4] @@ -275,424 +271,394 @@ Join condition: None Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(42) HashAggregate [codegen id : 15] +(42) SortAggregate Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] -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#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(43) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#35] -Results [2]: [c_customer_sk#29, 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 ssales#36] +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))#31] +Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] -(44) Filter [codegen id : 15] -Input [2]: [c_customer_sk#29, ssales#36] -Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) +(43) Filter [codegen id : 16] +Input [2]: [c_customer_sk#29, ssales#32] +Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) -(45) Project [codegen id : 15] +(44) Project [codegen id : 16] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#36] - -(46) Sort [codegen id : 15] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#29, ssales#32] -(47) SortMergeJoin [codegen id : 17] +(45) SortMergeJoin [codegen id : 18] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(48) Project [codegen id : 17] +(46) Project [codegen id : 18] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(49) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#39] +(47) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#35] -(50) BroadcastHashJoin [codegen id : 17] +(48) BroadcastHashJoin [codegen id : 18] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#39] +Right keys [1]: [d_date_sk#35] Join condition: None -(51) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#40] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#39] +(49) Project [codegen id : 18] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#36] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#35] -(52) Scan parquet default.web_sales -Output [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +(50) Scan parquet default.web_sales +Output [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#6)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +(51) ColumnarToRow [codegen id : 19] +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -(54) Exchange -Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Arguments: hashpartitioning(ws_item_sk#41, 5), ENSURE_REQUIREMENTS, [id=#46] +(52) Exchange +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: hashpartitioning(ws_item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#42] -(55) Sort [codegen id : 19] -Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Arguments: [ws_item_sk#41 ASC NULLS FIRST], false, 0 +(53) Sort [codegen id : 20] +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: [ws_item_sk#37 ASC NULLS FIRST], false, 0 -(56) ReusedExchange [Reuses operator id: 11] +(54) ReusedExchange [Reuses operator id: 11] Output [2]: [ss_item_sk#8, d_date#12] -(57) Sort [codegen id : 22] +(55) Sort [codegen id : 23] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(58) ReusedExchange [Reuses operator id: 16] +(56) ReusedExchange [Reuses operator id: 16] Output [2]: [i_item_sk#14, i_item_desc#15] -(59) Sort [codegen id : 24] +(57) Sort [codegen id : 25] Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 25] +(58) SortMergeJoin [codegen id : 26] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#14] Join condition: None -(61) Project [codegen id : 25] -Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#47] +(59) Project [codegen id : 26] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#43] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] -(62) HashAggregate [codegen id : 25] -Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#47] -Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] +(60) HashAggregate [codegen id : 26] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#43] +Keys [3]: [_groupingexpression#43, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] -Results [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] +Results [4]: [_groupingexpression#43, i_item_sk#14, d_date#12, count#19] -(63) HashAggregate [codegen id : 25] -Input [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] -Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] +(61) HashAggregate [codegen id : 26] +Input [4]: [_groupingexpression#43, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#43, i_item_sk#14, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] -(64) Filter [codegen id : 25] +(62) Filter [codegen id : 26] Input [2]: [item_sk#21, cnt#22] Condition : (cnt#22 > 4) -(65) Project [codegen id : 25] +(63) Project [codegen id : 26] Output [1]: [item_sk#21] Input [2]: [item_sk#21, cnt#22] -(66) Sort [codegen id : 25] +(64) Sort [codegen id : 26] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(67) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_item_sk#41] +(65) SortMergeJoin [codegen id : 27] +Left keys [1]: [ws_item_sk#37] Right keys [1]: [item_sk#21] Join condition: None -(68) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +(66) Project [codegen id : 27] +Output [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -(69) Exchange -Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#48] +(67) Exchange +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: hashpartitioning(ws_bill_customer_sk#38, 5), ENSURE_REQUIREMENTS, [id=#44] -(70) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 28] +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#38 ASC NULLS FIRST], false, 0 -(71) ReusedExchange [Reuses operator id: 33] +(69) ReusedExchange [Reuses operator id: 33] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(72) Sort [codegen id : 29] +(70) Sort [codegen id : 30] Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(73) ReusedExchange [Reuses operator id: 38] +(71) ReusedExchange [Reuses operator id: 38] Output [1]: [c_customer_sk#29] -(74) Sort [codegen id : 31] +(72) Sort [codegen id : 32] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 32] +(73) SortMergeJoin [codegen id : 33] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(76) Project [codegen id : 32] +(74) Project [codegen id : 33] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(77) HashAggregate [codegen id : 32] +(75) SortAggregate Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] -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#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(78) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#35] -Results [2]: [c_customer_sk#29, 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 ssales#36] +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))#31] +Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] -(79) Filter [codegen id : 32] -Input [2]: [c_customer_sk#29, ssales#36] -Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) +(76) Filter [codegen id : 34] +Input [2]: [c_customer_sk#29, ssales#32] +Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) -(80) Project [codegen id : 32] +(77) Project [codegen id : 34] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#36] - -(81) Sort [codegen id : 32] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#29, ssales#32] -(82) SortMergeJoin [codegen id : 34] -Left keys [1]: [ws_bill_customer_sk#42] +(78) SortMergeJoin [codegen id : 36] +Left keys [1]: [ws_bill_customer_sk#38] Right keys [1]: [c_customer_sk#29] Join condition: None -(83) Project [codegen id : 34] -Output [3]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +(79) Project [codegen id : 36] +Output [3]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -(84) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#49] +(80) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#45] -(85) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#49] +(81) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#45] Join condition: None -(86) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#50] -Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#49] +(82) Project [codegen id : 36] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#39 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#40 as decimal(12,2)))), DecimalType(18,2), true) AS sales#46] +Input [4]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, d_date_sk#45] -(87) Union +(83) Union -(88) HashAggregate [codegen id : 35] -Input [1]: [sales#40] +(84) HashAggregate [codegen id : 37] +Input [1]: [sales#36] Keys: [] -Functions [1]: [partial_sum(sales#40)] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [2]: [sum#53, isEmpty#54] +Functions [1]: [partial_sum(sales#36)] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [2]: [sum#49, isEmpty#50] -(89) Exchange -Input [2]: [sum#53, isEmpty#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#55] +(85) Exchange +Input [2]: [sum#49, isEmpty#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] -(90) HashAggregate [codegen id : 36] -Input [2]: [sum#53, isEmpty#54] +(86) HashAggregate [codegen id : 38] +Input [2]: [sum#49, isEmpty#50] Keys: [] -Functions [1]: [sum(sales#40)] -Aggregate Attributes [1]: [sum(sales#40)#56] -Results [1]: [sum(sales#40)#56 AS sum(sales)#57] +Functions [1]: [sum(sales#36)] +Aggregate Attributes [1]: [sum(sales#36)#52] +Results [1]: [sum(sales#36)#52 AS sum(sales)#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * Project (94) - +- * Filter (93) - +- * ColumnarToRow (92) - +- Scan parquet default.date_dim (91) +BroadcastExchange (91) ++- * Project (90) + +- * Filter (89) + +- * ColumnarToRow (88) + +- Scan parquet default.date_dim (87) -(91) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#58, d_moy#59] +(87) Scan parquet default.date_dim +Output [3]: [d_date_sk#35, d_year#54, d_moy#55] 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 -(92) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#58, d_moy#59] +(88) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#35, d_year#54, d_moy#55] -(93) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#58, d_moy#59] -Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#39)) +(89) Filter [codegen id : 1] +Input [3]: [d_date_sk#35, d_year#54, d_moy#55] +Condition : ((((isnotnull(d_year#54) AND isnotnull(d_moy#55)) AND (d_year#54 = 2000)) AND (d_moy#55 = 2)) AND isnotnull(d_date_sk#35)) -(94) Project [codegen id : 1] -Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#58, d_moy#59] +(90) Project [codegen id : 1] +Output [1]: [d_date_sk#35] +Input [3]: [d_date_sk#35, d_year#54, d_moy#55] -(95) BroadcastExchange -Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +(91) BroadcastExchange +Input [1]: [d_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (100) -+- * Project (99) - +- * Filter (98) - +- * ColumnarToRow (97) - +- Scan parquet default.date_dim (96) +BroadcastExchange (96) ++- * Project (95) + +- * Filter (94) + +- * ColumnarToRow (93) + +- Scan parquet default.date_dim (92) -(96) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#61] +(92) Scan parquet default.date_dim +Output [3]: [d_date_sk#11, d_date#12, d_year#57] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(97) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#61] +(93) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#57] -(98) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#61] -Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +(94) Filter [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#57] +Condition : (d_year#57 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(99) Project [codegen id : 1] +(95) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#61] +Input [3]: [d_date_sk#11, d_date#12, d_year#57] -(100) BroadcastExchange +(96) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] - -Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* HashAggregate (117) -+- Exchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * HashAggregate (113) - +- * Project (112) - +- * SortMergeJoin Inner (111) - :- * Sort (108) - : +- Exchange (107) - : +- * Project (106) - : +- * BroadcastHashJoin Inner BuildRight (105) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- ReusedExchange (104) - +- * Sort (110) - +- ReusedExchange (109) - - -(101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] + +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#33, [id=#34] +* HashAggregate (112) ++- Exchange (111) + +- * HashAggregate (110) + +- SortAggregate (109) + +- * Project (108) + +- * SortMergeJoin Inner (107) + :- * Sort (104) + : +- Exchange (103) + : +- * Project (102) + : +- * BroadcastHashJoin Inner BuildRight (101) + : :- * Filter (99) + : : +- * ColumnarToRow (98) + : : +- Scan parquet default.store_sales (97) + : +- ReusedExchange (100) + +- * Sort (106) + +- ReusedExchange (105) + + +(97) Scan parquet default.store_sales +Output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] +(98) ColumnarToRow [codegen id : 2] +Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] -(103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] -Condition : isnotnull(ss_customer_sk#63) +(99) Filter [codegen id : 2] +Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Condition : isnotnull(ss_customer_sk#59) -(104) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#68] +(100) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#64] -(105) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#66] -Right keys [1]: [d_date_sk#68] +(101) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#62] +Right keys [1]: [d_date_sk#64] Join condition: None -(106) Project [codegen id : 2] -Output [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] -Input [5]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, d_date_sk#68] +(102) Project [codegen id : 2] +Output [3]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61] +Input [5]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, d_date_sk#64] -(107) Exchange -Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] -Arguments: hashpartitioning(ss_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +(103) Exchange +Input [3]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61] +Arguments: hashpartitioning(ss_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] -(108) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] -Arguments: [ss_customer_sk#63 ASC NULLS FIRST], false, 0 +(104) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61] +Arguments: [ss_customer_sk#59 ASC NULLS FIRST], false, 0 -(109) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#70] +(105) ReusedExchange [Reuses operator id: 38] +Output [1]: [c_customer_sk#66] -(110) Sort [codegen id : 5] -Input [1]: [c_customer_sk#70] -Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 +(106) Sort [codegen id : 5] +Input [1]: [c_customer_sk#66] +Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 -(111) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#63] -Right keys [1]: [c_customer_sk#70] +(107) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#59] +Right keys [1]: [c_customer_sk#66] Join condition: None -(112) Project [codegen id : 6] -Output [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] -Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, c_customer_sk#70] - -(113) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] -Keys [1]: [c_customer_sk#70] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#71, isEmpty#72] -Results [3]: [c_customer_sk#70, sum#73, isEmpty#74] - -(114) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#70, sum#73, isEmpty#74] -Keys [1]: [c_customer_sk#70] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75 AS csales#76] - -(115) HashAggregate [codegen id : 6] -Input [1]: [csales#76] +(108) Project [codegen id : 6] +Output [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#66] +Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, c_customer_sk#66] + +(109) SortAggregate +Input [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#66] +Keys [1]: [c_customer_sk#66] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#61 as decimal(12,2)))), DecimalType(18,2), true))#67] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#61 as decimal(12,2)))), DecimalType(18,2), true))#67 AS csales#68] + +(110) HashAggregate [codegen id : 7] +Input [1]: [csales#68] Keys: [] -Functions [1]: [partial_max(csales#76)] -Aggregate Attributes [1]: [max#77] -Results [1]: [max#78] +Functions [1]: [partial_max(csales#68)] +Aggregate Attributes [1]: [max#69] +Results [1]: [max#70] -(116) Exchange -Input [1]: [max#78] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#79] +(111) Exchange +Input [1]: [max#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#71] -(117) HashAggregate [codegen id : 7] -Input [1]: [max#78] +(112) HashAggregate [codegen id : 8] +Input [1]: [max#70] Keys: [] -Functions [1]: [max(csales#76)] -Aggregate Attributes [1]: [max(csales#76)#80] -Results [1]: [max(csales#76)#80 AS tpcds_cmax#81] +Functions [1]: [max(csales#68)] +Aggregate Attributes [1]: [max(csales#68)#72] +Results [1]: [max(csales#68)#72 AS tpcds_cmax#73] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 -BroadcastExchange (122) -+- * Project (121) - +- * Filter (120) - +- * ColumnarToRow (119) - +- Scan parquet default.date_dim (118) +Subquery:4 Hosting operator id = 97 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 +BroadcastExchange (117) ++- * Project (116) + +- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet default.date_dim (113) -(118) Scan parquet default.date_dim -Output [2]: [d_date_sk#68, d_year#82] +(113) Scan parquet default.date_dim +Output [2]: [d_date_sk#64, d_year#74] 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 -(119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#68, d_year#82] +(114) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#64, d_year#74] -(120) Filter [codegen id : 1] -Input [2]: [d_date_sk#68, d_year#82] -Condition : (d_year#82 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) +(115) Filter [codegen id : 1] +Input [2]: [d_date_sk#64, d_year#74] +Condition : (d_year#74 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) -(121) Project [codegen id : 1] -Output [1]: [d_date_sk#68] -Input [2]: [d_date_sk#68, d_year#82] +(116) Project [codegen id : 1] +Output [1]: [d_date_sk#64] +Input [2]: [d_date_sk#64, d_year#74] -(122) BroadcastExchange -Input [1]: [d_date_sk#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +(117) BroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:6 Hosting operator id = 76 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] 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 17377b91326fd..5c2a89277801a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -1,12 +1,12 @@ -WholeStageCodegen (36) +WholeStageCodegen (38) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 - WholeStageCodegen (35) + WholeStageCodegen (37) HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (17) + WholeStageCodegen (18) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk] @@ -78,19 +78,19 @@ WholeStageCodegen (36) InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (15) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (7) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #10 - 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] + WholeStageCodegen (16) + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #10 + WholeStageCodegen (7) + HashAggregate [csales] [max,max] + InputAdapter + SortAggregate [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] + WholeStageCodegen (6) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter @@ -120,8 +120,9 @@ WholeStageCodegen (36) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 - 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)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + InputAdapter + SortAggregate [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)),ssales,sum,isEmpty] + WholeStageCodegen (15) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter @@ -147,31 +148,31 @@ WholeStageCodegen (36) Scan parquet default.customer [c_customer_sk] InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (34) + WholeStageCodegen (36) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (27) + WholeStageCodegen (28) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #13 - WholeStageCodegen (26) + WholeStageCodegen (27) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_item_sk,item_sk] InputAdapter - WholeStageCodegen (19) + WholeStageCodegen (20) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #14 - WholeStageCodegen (18) + WholeStageCodegen (19) ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (26) Sort [item_sk] Project [item_sk] Filter [cnt] @@ -180,32 +181,32 @@ WholeStageCodegen (36) Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (22) + WholeStageCodegen (23) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk,d_date] #5 InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (25) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 InputAdapter - WholeStageCodegen (32) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + WholeStageCodegen (34) + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + InputAdapter + SortAggregate [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)),ssales,sum,isEmpty] + WholeStageCodegen (33) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (30) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (32) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 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 638f5ec3ded62..735457aa23567 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,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (129) -+- Union (128) - :- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * SortMergeJoin Inner (69) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * SortMergeJoin LeftSemi (48) +TakeOrderedAndProject (121) ++- Union (120) + :- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * Project (66) + : +- * SortMergeJoin Inner (65) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * SortMergeJoin LeftSemi (46) : : : :- * Sort (29) : : : : +- Exchange (28) : : : : +- * Project (27) @@ -38,96 +38,88 @@ TakeOrderedAndProject (129) : : : : +- * Filter (16) : : : : +- * ColumnarToRow (15) : : : : +- Scan parquet default.item (14) - : : : +- * Sort (47) - : : : +- * Project (46) - : : : +- * Filter (45) - : : : +- * HashAggregate (44) - : : : +- * HashAggregate (43) - : : : +- * Project (42) - : : : +- * SortMergeJoin Inner (41) - : : : :- * Sort (35) - : : : : +- Exchange (34) - : : : : +- * Project (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.store_sales (30) - : : : +- * Sort (40) - : : : +- Exchange (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.customer (36) - : : +- ReusedExchange (49) - : +- * SortMergeJoin LeftSemi (68) - : :- * Sort (56) - : : +- Exchange (55) - : : +- * Filter (54) - : : +- * ColumnarToRow (53) - : : +- Scan parquet default.customer (52) - : +- * Sort (67) - : +- * Project (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- * HashAggregate (63) - : +- * Project (62) - : +- * SortMergeJoin Inner (61) - : :- * Sort (58) - : : +- ReusedExchange (57) - : +- * Sort (60) - : +- ReusedExchange (59) - +- * HashAggregate (127) - +- Exchange (126) - +- * HashAggregate (125) - +- * Project (124) - +- * SortMergeJoin Inner (123) - :- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * SortMergeJoin LeftSemi (105) - : : :- * Sort (93) - : : : +- Exchange (92) - : : : +- * Project (91) - : : : +- * SortMergeJoin LeftSemi (90) - : : : :- * Sort (78) - : : : : +- Exchange (77) - : : : : +- * Filter (76) - : : : : +- * ColumnarToRow (75) - : : : : +- Scan parquet default.web_sales (74) - : : : +- * Sort (89) - : : : +- * Project (88) - : : : +- * Filter (87) - : : : +- * HashAggregate (86) - : : : +- * HashAggregate (85) - : : : +- * Project (84) - : : : +- * SortMergeJoin Inner (83) - : : : :- * Sort (80) - : : : : +- ReusedExchange (79) - : : : +- * Sort (82) - : : : +- ReusedExchange (81) - : : +- * Sort (104) - : : +- * Project (103) - : : +- * Filter (102) - : : +- * HashAggregate (101) - : : +- * HashAggregate (100) - : : +- * Project (99) - : : +- * SortMergeJoin Inner (98) - : : :- * Sort (95) - : : : +- ReusedExchange (94) - : : +- * Sort (97) - : : +- ReusedExchange (96) - : +- ReusedExchange (106) - +- * SortMergeJoin LeftSemi (122) - :- * Sort (110) - : +- ReusedExchange (109) - +- * Sort (121) - +- * Project (120) - +- * Filter (119) - +- * HashAggregate (118) - +- * HashAggregate (117) - +- * Project (116) - +- * SortMergeJoin Inner (115) - :- * Sort (112) - : +- ReusedExchange (111) - +- * Sort (114) - +- ReusedExchange (113) + : : : +- * Project (45) + : : : +- * Filter (44) + : : : +- SortAggregate (43) + : : : +- * Project (42) + : : : +- * SortMergeJoin Inner (41) + : : : :- * Sort (35) + : : : : +- Exchange (34) + : : : : +- * Project (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.store_sales (30) + : : : +- * Sort (40) + : : : +- Exchange (39) + : : : +- * Filter (38) + : : : +- * ColumnarToRow (37) + : : : +- Scan parquet default.customer (36) + : : +- ReusedExchange (47) + : +- * SortMergeJoin LeftSemi (64) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * Filter (52) + : : +- * ColumnarToRow (51) + : : +- Scan parquet default.customer (50) + : +- * Project (63) + : +- * Filter (62) + : +- SortAggregate (61) + : +- * Project (60) + : +- * SortMergeJoin Inner (59) + : :- * Sort (56) + : : +- ReusedExchange (55) + : +- * Sort (58) + : +- ReusedExchange (57) + +- * HashAggregate (119) + +- Exchange (118) + +- * HashAggregate (117) + +- * Project (116) + +- * SortMergeJoin Inner (115) + :- * Project (102) + : +- * BroadcastHashJoin Inner BuildRight (101) + : :- * SortMergeJoin LeftSemi (99) + : : :- * Sort (89) + : : : +- Exchange (88) + : : : +- * Project (87) + : : : +- * SortMergeJoin LeftSemi (86) + : : : :- * Sort (74) + : : : : +- Exchange (73) + : : : : +- * Filter (72) + : : : : +- * ColumnarToRow (71) + : : : : +- Scan parquet default.web_sales (70) + : : : +- * Sort (85) + : : : +- * Project (84) + : : : +- * Filter (83) + : : : +- * HashAggregate (82) + : : : +- * HashAggregate (81) + : : : +- * Project (80) + : : : +- * SortMergeJoin Inner (79) + : : : :- * Sort (76) + : : : : +- ReusedExchange (75) + : : : +- * Sort (78) + : : : +- ReusedExchange (77) + : : +- * Project (98) + : : +- * Filter (97) + : : +- SortAggregate (96) + : : +- * Project (95) + : : +- * SortMergeJoin Inner (94) + : : :- * Sort (91) + : : : +- ReusedExchange (90) + : : +- * Sort (93) + : : +- ReusedExchange (92) + : +- ReusedExchange (100) + +- * SortMergeJoin LeftSemi (114) + :- * Sort (104) + : +- ReusedExchange (103) + +- * Project (113) + +- * Filter (112) + +- SortAggregate (111) + +- * Project (110) + +- * SortMergeJoin Inner (109) + :- * Sort (106) + : +- ReusedExchange (105) + +- * Sort (108) + +- ReusedExchange (107) (1) Scan parquet default.catalog_sales @@ -168,7 +160,7 @@ Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(9) ReusedExchange [Reuses operator id: 139] +(9) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#11, d_date#12] (10) BroadcastHashJoin [codegen id : 4] @@ -319,602 +311,550 @@ Join condition: None Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(43) HashAggregate [codegen id : 15] +(43) SortAggregate Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] -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#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(44) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#35] -Results [2]: [c_customer_sk#29, 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 ssales#36] +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))#31] +Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] -(45) Filter [codegen id : 15] -Input [2]: [c_customer_sk#29, ssales#36] -Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) +(44) Filter [codegen id : 16] +Input [2]: [c_customer_sk#29, ssales#32] +Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) -(46) Project [codegen id : 15] +(45) Project [codegen id : 16] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#36] - -(47) Sort [codegen id : 15] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#29, ssales#32] -(48) SortMergeJoin [codegen id : 17] +(46) SortMergeJoin [codegen id : 18] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(49) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#39] +(47) ReusedExchange [Reuses operator id: 126] +Output [1]: [d_date_sk#35] -(50) BroadcastHashJoin [codegen id : 17] +(48) BroadcastHashJoin [codegen id : 18] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#39] +Right keys [1]: [d_date_sk#35] Join condition: None -(51) Project [codegen id : 17] +(49) Project [codegen id : 18] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#39] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#35] -(52) Scan parquet default.customer -Output [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] +(50) Scan parquet default.customer +Output [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] +(51) ColumnarToRow [codegen id : 19] +Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] -(54) Filter [codegen id : 18] -Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] -Condition : isnotnull(c_customer_sk#40) +(52) Filter [codegen id : 19] +Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] +Condition : isnotnull(c_customer_sk#36) -(55) Exchange -Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] -Arguments: hashpartitioning(c_customer_sk#40, 5), ENSURE_REQUIREMENTS, [id=#43] +(53) Exchange +Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] +Arguments: hashpartitioning(c_customer_sk#36, 5), ENSURE_REQUIREMENTS, [id=#39] -(56) Sort [codegen id : 19] -Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] -Arguments: [c_customer_sk#40 ASC NULLS FIRST], false, 0 +(54) Sort [codegen id : 20] +Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] +Arguments: [c_customer_sk#36 ASC NULLS FIRST], false, 0 -(57) ReusedExchange [Reuses operator id: 34] +(55) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(58) Sort [codegen id : 21] +(56) Sort [codegen id : 22] Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(59) ReusedExchange [Reuses operator id: 39] +(57) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#29] -(60) Sort [codegen id : 23] +(58) Sort [codegen id : 24] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(61) SortMergeJoin [codegen id : 24] +(59) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(62) Project [codegen id : 24] +(60) Project [codegen id : 25] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(63) HashAggregate [codegen id : 24] +(61) SortAggregate Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] -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#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(64) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#35] -Results [2]: [c_customer_sk#29, 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 ssales#36] +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))#31] +Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] -(65) Filter [codegen id : 24] -Input [2]: [c_customer_sk#29, ssales#36] -Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) +(62) Filter [codegen id : 26] +Input [2]: [c_customer_sk#29, ssales#32] +Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) -(66) Project [codegen id : 24] +(63) Project [codegen id : 26] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#36] - -(67) Sort [codegen id : 24] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#29, ssales#32] -(68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_customer_sk#40] +(64) SortMergeJoin [codegen id : 27] +Left keys [1]: [c_customer_sk#36] Right keys [1]: [c_customer_sk#29] Join condition: None -(69) SortMergeJoin [codegen id : 26] +(65) SortMergeJoin [codegen id : 28] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#40] +Right keys [1]: [c_customer_sk#36] Join condition: None -(70) Project [codegen id : 26] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#41, c_last_name#42] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#40, c_first_name#41, c_last_name#42] +(66) Project [codegen id : 28] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#37, c_last_name#38] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#36, c_first_name#37, c_last_name#38] -(71) HashAggregate [codegen id : 26] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#41, c_last_name#42] -Keys [2]: [c_last_name#42, c_first_name#41] +(67) HashAggregate [codegen id : 28] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#37, c_last_name#38] +Keys [2]: [c_last_name#38, c_first_name#37] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [4]: [c_last_name#42, c_first_name#41, sum#46, isEmpty#47] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [4]: [c_last_name#38, c_first_name#37, sum#42, isEmpty#43] -(72) Exchange -Input [4]: [c_last_name#42, c_first_name#41, sum#46, isEmpty#47] -Arguments: hashpartitioning(c_last_name#42, c_first_name#41, 5), ENSURE_REQUIREMENTS, [id=#48] +(68) Exchange +Input [4]: [c_last_name#38, c_first_name#37, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#38, c_first_name#37, 5), ENSURE_REQUIREMENTS, [id=#44] -(73) HashAggregate [codegen id : 27] -Input [4]: [c_last_name#42, c_first_name#41, sum#46, isEmpty#47] -Keys [2]: [c_last_name#42, c_first_name#41] +(69) HashAggregate [codegen id : 29] +Input [4]: [c_last_name#38, c_first_name#37, sum#42, isEmpty#43] +Keys [2]: [c_last_name#38, c_first_name#37] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#49] -Results [3]: [c_last_name#42, c_first_name#41, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#49 AS sales#50] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] +Results [3]: [c_last_name#38, c_first_name#37, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45 AS sales#46] -(74) Scan parquet default.web_sales -Output [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +(70) Scan parquet default.web_sales +Output [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#55), dynamicpruningexpression(ws_sold_date_sk#55 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 28] -Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +(71) ColumnarToRow [codegen id : 30] +Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -(76) Filter [codegen id : 28] -Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Condition : isnotnull(ws_bill_customer_sk#52) +(72) Filter [codegen id : 30] +Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#48) -(77) Exchange -Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Arguments: hashpartitioning(ws_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] +(73) Exchange +Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +Arguments: hashpartitioning(ws_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#52] -(78) Sort [codegen id : 29] -Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Arguments: [ws_item_sk#51 ASC NULLS FIRST], false, 0 +(74) Sort [codegen id : 31] +Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +Arguments: [ws_item_sk#47 ASC NULLS FIRST], false, 0 -(79) ReusedExchange [Reuses operator id: 12] +(75) ReusedExchange [Reuses operator id: 12] Output [2]: [ss_item_sk#8, d_date#12] -(80) Sort [codegen id : 32] +(76) Sort [codegen id : 34] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(81) ReusedExchange [Reuses operator id: 17] +(77) ReusedExchange [Reuses operator id: 17] Output [2]: [i_item_sk#14, i_item_desc#15] -(82) Sort [codegen id : 34] +(78) Sort [codegen id : 36] Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 -(83) SortMergeJoin [codegen id : 35] +(79) SortMergeJoin [codegen id : 37] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#14] Join condition: None -(84) Project [codegen id : 35] -Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#57] +(80) Project [codegen id : 37] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#53] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] -(85) HashAggregate [codegen id : 35] -Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#57] -Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] +(81) HashAggregate [codegen id : 37] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#53] +Keys [3]: [_groupingexpression#53, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] -Results [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] +Results [4]: [_groupingexpression#53, i_item_sk#14, d_date#12, count#19] -(86) HashAggregate [codegen id : 35] -Input [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] -Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] +(82) HashAggregate [codegen id : 37] +Input [4]: [_groupingexpression#53, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#53, i_item_sk#14, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] -(87) Filter [codegen id : 35] +(83) Filter [codegen id : 37] Input [2]: [item_sk#21, cnt#22] Condition : (cnt#22 > 4) -(88) Project [codegen id : 35] +(84) Project [codegen id : 37] Output [1]: [item_sk#21] Input [2]: [item_sk#21, cnt#22] -(89) Sort [codegen id : 35] +(85) Sort [codegen id : 37] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 36] -Left keys [1]: [ws_item_sk#51] +(86) SortMergeJoin [codegen id : 38] +Left keys [1]: [ws_item_sk#47] Right keys [1]: [item_sk#21] Join condition: None -(91) Project [codegen id : 36] -Output [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +(87) Project [codegen id : 38] +Output [4]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -(92) Exchange -Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Arguments: hashpartitioning(ws_bill_customer_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] +(88) Exchange +Input [4]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +Arguments: hashpartitioning(ws_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] -(93) Sort [codegen id : 37] -Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Arguments: [ws_bill_customer_sk#52 ASC NULLS FIRST], false, 0 +(89) Sort [codegen id : 39] +Input [4]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +Arguments: [ws_bill_customer_sk#48 ASC NULLS FIRST], false, 0 -(94) ReusedExchange [Reuses operator id: 34] +(90) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(95) Sort [codegen id : 39] +(91) Sort [codegen id : 41] Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(96) ReusedExchange [Reuses operator id: 39] +(92) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#29] -(97) Sort [codegen id : 41] +(93) Sort [codegen id : 43] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin [codegen id : 42] +(94) SortMergeJoin [codegen id : 44] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(99) Project [codegen id : 42] +(95) Project [codegen id : 44] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(100) HashAggregate [codegen id : 42] +(96) SortAggregate Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] -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#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(101) HashAggregate [codegen id : 42] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#35] -Results [2]: [c_customer_sk#29, 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 ssales#36] +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))#31] +Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] -(102) Filter [codegen id : 42] -Input [2]: [c_customer_sk#29, ssales#36] -Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) +(97) Filter [codegen id : 45] +Input [2]: [c_customer_sk#29, ssales#32] +Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) -(103) Project [codegen id : 42] +(98) Project [codegen id : 45] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#36] +Input [2]: [c_customer_sk#29, ssales#32] -(104) Sort [codegen id : 42] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 - -(105) SortMergeJoin [codegen id : 44] -Left keys [1]: [ws_bill_customer_sk#52] +(99) SortMergeJoin [codegen id : 47] +Left keys [1]: [ws_bill_customer_sk#48] Right keys [1]: [c_customer_sk#29] Join condition: None -(106) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#59] +(100) ReusedExchange [Reuses operator id: 126] +Output [1]: [d_date_sk#55] -(107) BroadcastHashJoin [codegen id : 44] -Left keys [1]: [ws_sold_date_sk#55] -Right keys [1]: [d_date_sk#59] +(101) BroadcastHashJoin [codegen id : 47] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#55] Join condition: None -(108) Project [codegen id : 44] -Output [3]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54] -Input [5]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55, d_date_sk#59] +(102) Project [codegen id : 47] +Output [3]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50] +Input [5]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51, d_date_sk#55] -(109) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] +(103) ReusedExchange [Reuses operator id: 53] +Output [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] -(110) Sort [codegen id : 46] -Input [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] -Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 +(104) Sort [codegen id : 49] +Input [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] +Arguments: [c_customer_sk#56 ASC NULLS FIRST], false, 0 -(111) ReusedExchange [Reuses operator id: 34] +(105) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(112) Sort [codegen id : 48] +(106) Sort [codegen id : 51] Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(113) ReusedExchange [Reuses operator id: 39] +(107) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#29] -(114) Sort [codegen id : 50] +(108) Sort [codegen id : 53] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(115) SortMergeJoin [codegen id : 51] +(109) SortMergeJoin [codegen id : 54] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(116) Project [codegen id : 51] +(110) Project [codegen id : 54] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(117) HashAggregate [codegen id : 51] +(111) SortAggregate Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] -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#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] - -(118) HashAggregate [codegen id : 51] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#35] -Results [2]: [c_customer_sk#29, 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 ssales#36] +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))#31] +Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] -(119) Filter [codegen id : 51] -Input [2]: [c_customer_sk#29, ssales#36] -Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) +(112) Filter [codegen id : 55] +Input [2]: [c_customer_sk#29, ssales#32] +Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) -(120) Project [codegen id : 51] +(113) Project [codegen id : 55] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#36] - -(121) Sort [codegen id : 51] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#29, ssales#32] -(122) SortMergeJoin [codegen id : 52] -Left keys [1]: [c_customer_sk#60] +(114) SortMergeJoin [codegen id : 56] +Left keys [1]: [c_customer_sk#56] Right keys [1]: [c_customer_sk#29] Join condition: None -(123) SortMergeJoin [codegen id : 53] -Left keys [1]: [ws_bill_customer_sk#52] -Right keys [1]: [c_customer_sk#60] +(115) SortMergeJoin [codegen id : 57] +Left keys [1]: [ws_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#56] Join condition: None -(124) Project [codegen id : 53] -Output [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] -Input [6]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, c_customer_sk#60, c_first_name#61, c_last_name#62] +(116) Project [codegen id : 57] +Output [4]: [ws_quantity#49, ws_list_price#50, c_first_name#57, c_last_name#58] +Input [6]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, c_customer_sk#56, c_first_name#57, c_last_name#58] -(125) HashAggregate [codegen id : 53] -Input [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] -Keys [2]: [c_last_name#62, c_first_name#61] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +(117) HashAggregate [codegen id : 57] +Input [4]: [ws_quantity#49, ws_list_price#50, c_first_name#57, c_last_name#58] +Keys [2]: [c_last_name#58, c_first_name#57] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#59, isEmpty#60] +Results [4]: [c_last_name#58, c_first_name#57, sum#61, isEmpty#62] -(126) Exchange -Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] -Arguments: hashpartitioning(c_last_name#62, c_first_name#61, 5), ENSURE_REQUIREMENTS, [id=#67] +(118) Exchange +Input [4]: [c_last_name#58, c_first_name#57, sum#61, isEmpty#62] +Arguments: hashpartitioning(c_last_name#58, c_first_name#57, 5), ENSURE_REQUIREMENTS, [id=#63] -(127) HashAggregate [codegen id : 54] -Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] -Keys [2]: [c_last_name#62, c_first_name#61] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68] -Results [3]: [c_last_name#62, c_first_name#61, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68 AS sales#69] +(119) HashAggregate [codegen id : 58] +Input [4]: [c_last_name#58, c_first_name#57, sum#61, isEmpty#62] +Keys [2]: [c_last_name#58, c_first_name#57] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))#64] +Results [3]: [c_last_name#58, c_first_name#57, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#65] -(128) Union +(120) Union -(129) TakeOrderedAndProject -Input [3]: [c_last_name#42, c_first_name#41, sales#50] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#50] +(121) TakeOrderedAndProject +Input [3]: [c_last_name#38, c_first_name#37, sales#46] +Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, sales#46 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, sales#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (134) -+- * Project (133) - +- * Filter (132) - +- * ColumnarToRow (131) - +- Scan parquet default.date_dim (130) +BroadcastExchange (126) ++- * Project (125) + +- * Filter (124) + +- * ColumnarToRow (123) + +- Scan parquet default.date_dim (122) -(130) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#70, d_moy#71] +(122) Scan parquet default.date_dim +Output [3]: [d_date_sk#35, d_year#66, d_moy#67] 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 -(131) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#70, d_moy#71] +(123) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#35, d_year#66, d_moy#67] -(132) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#70, d_moy#71] -Condition : ((((isnotnull(d_year#70) AND isnotnull(d_moy#71)) AND (d_year#70 = 2000)) AND (d_moy#71 = 2)) AND isnotnull(d_date_sk#39)) +(124) Filter [codegen id : 1] +Input [3]: [d_date_sk#35, d_year#66, d_moy#67] +Condition : ((((isnotnull(d_year#66) AND isnotnull(d_moy#67)) AND (d_year#66 = 2000)) AND (d_moy#67 = 2)) AND isnotnull(d_date_sk#35)) -(133) Project [codegen id : 1] -Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#70, d_moy#71] +(125) Project [codegen id : 1] +Output [1]: [d_date_sk#35] +Input [3]: [d_date_sk#35, d_year#66, d_moy#67] -(134) BroadcastExchange -Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +(126) BroadcastExchange +Input [1]: [d_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (139) -+- * Project (138) - +- * Filter (137) - +- * ColumnarToRow (136) - +- Scan parquet default.date_dim (135) +BroadcastExchange (131) ++- * Project (130) + +- * Filter (129) + +- * ColumnarToRow (128) + +- Scan parquet default.date_dim (127) -(135) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#73] +(127) Scan parquet default.date_dim +Output [3]: [d_date_sk#11, d_date#12, d_year#69] 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 -(136) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#73] +(128) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#69] -(137) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#73] -Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +(129) Filter [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#69] +Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(138) Project [codegen id : 1] +(130) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#73] +Input [3]: [d_date_sk#11, d_date#12, d_year#69] -(139) BroadcastExchange +(131) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#74] - -Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* HashAggregate (156) -+- Exchange (155) - +- * HashAggregate (154) - +- * HashAggregate (153) - +- * HashAggregate (152) - +- * Project (151) - +- * SortMergeJoin Inner (150) - :- * Sort (147) - : +- Exchange (146) - : +- * Project (145) - : +- * BroadcastHashJoin Inner BuildRight (144) - : :- * Filter (142) - : : +- * ColumnarToRow (141) - : : +- Scan parquet default.store_sales (140) - : +- ReusedExchange (143) - +- * Sort (149) - +- ReusedExchange (148) - - -(140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] + +Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#33, [id=#34] +* HashAggregate (147) ++- Exchange (146) + +- * HashAggregate (145) + +- SortAggregate (144) + +- * Project (143) + +- * SortMergeJoin Inner (142) + :- * Sort (139) + : +- Exchange (138) + : +- * Project (137) + : +- * BroadcastHashJoin Inner BuildRight (136) + : :- * Filter (134) + : : +- * ColumnarToRow (133) + : : +- Scan parquet default.store_sales (132) + : +- ReusedExchange (135) + +- * Sort (141) + +- ReusedExchange (140) + + +(132) Scan parquet default.store_sales +Output [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#74), dynamicpruningexpression(ss_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] +(133) ColumnarToRow [codegen id : 2] +Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74] -(142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_customer_sk#75) +(134) Filter [codegen id : 2] +Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74] +Condition : isnotnull(ss_customer_sk#71) -(143) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#80] +(135) ReusedExchange [Reuses operator id: 152] +Output [1]: [d_date_sk#76] -(144) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#80] +(136) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#74] +Right keys [1]: [d_date_sk#76] Join condition: None -(145) Project [codegen id : 2] -Output [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] -Input [5]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78, d_date_sk#80] +(137) Project [codegen id : 2] +Output [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] +Input [5]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74, d_date_sk#76] -(146) Exchange -Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] -Arguments: hashpartitioning(ss_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] +(138) Exchange +Input [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] +Arguments: hashpartitioning(ss_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#77] -(147) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] -Arguments: [ss_customer_sk#75 ASC NULLS FIRST], false, 0 +(139) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] +Arguments: [ss_customer_sk#71 ASC NULLS FIRST], false, 0 -(148) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#82] +(140) ReusedExchange [Reuses operator id: 39] +Output [1]: [c_customer_sk#78] -(149) Sort [codegen id : 5] -Input [1]: [c_customer_sk#82] -Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 +(141) Sort [codegen id : 5] +Input [1]: [c_customer_sk#78] +Arguments: [c_customer_sk#78 ASC NULLS FIRST], false, 0 -(150) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#75] -Right keys [1]: [c_customer_sk#82] +(142) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#71] +Right keys [1]: [c_customer_sk#78] Join condition: None -(151) Project [codegen id : 6] -Output [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] -Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, c_customer_sk#82] - -(152) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] -Keys [1]: [c_customer_sk#82] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [3]: [c_customer_sk#82, sum#85, isEmpty#86] - -(153) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#82, sum#85, isEmpty#86] -Keys [1]: [c_customer_sk#82] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] - -(154) HashAggregate [codegen id : 6] -Input [1]: [csales#88] +(143) Project [codegen id : 6] +Output [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#78] +Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, c_customer_sk#78] + +(144) SortAggregate +Input [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#78] +Keys [1]: [c_customer_sk#78] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79 AS csales#80] + +(145) HashAggregate [codegen id : 7] +Input [1]: [csales#80] Keys: [] -Functions [1]: [partial_max(csales#88)] -Aggregate Attributes [1]: [max#89] -Results [1]: [max#90] +Functions [1]: [partial_max(csales#80)] +Aggregate Attributes [1]: [max#81] +Results [1]: [max#82] -(155) Exchange -Input [1]: [max#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +(146) Exchange +Input [1]: [max#82] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] -(156) HashAggregate [codegen id : 7] -Input [1]: [max#90] +(147) HashAggregate [codegen id : 8] +Input [1]: [max#82] Keys: [] -Functions [1]: [max(csales#88)] -Aggregate Attributes [1]: [max(csales#88)#92] -Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] +Functions [1]: [max(csales#80)] +Aggregate Attributes [1]: [max(csales#80)#84] +Results [1]: [max(csales#80)#84 AS tpcds_cmax#85] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 -BroadcastExchange (161) -+- * Project (160) - +- * Filter (159) - +- * ColumnarToRow (158) - +- Scan parquet default.date_dim (157) +Subquery:4 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (152) ++- * Project (151) + +- * Filter (150) + +- * ColumnarToRow (149) + +- Scan parquet default.date_dim (148) -(157) Scan parquet default.date_dim -Output [2]: [d_date_sk#80, d_year#94] +(148) Scan parquet default.date_dim +Output [2]: [d_date_sk#76, d_year#86] 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 -(158) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#80, d_year#94] +(149) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#76, d_year#86] -(159) Filter [codegen id : 1] -Input [2]: [d_date_sk#80, d_year#94] -Condition : (d_year#94 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#80)) +(150) Filter [codegen id : 1] +Input [2]: [d_date_sk#76, d_year#86] +Condition : (d_year#86 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#76)) -(160) Project [codegen id : 1] -Output [1]: [d_date_sk#80] -Input [2]: [d_date_sk#80, d_year#94] +(151) Project [codegen id : 1] +Output [1]: [d_date_sk#76] +Input [2]: [d_date_sk#76, d_year#86] -(161) BroadcastExchange -Input [1]: [d_date_sk#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#95] +(152) BroadcastExchange +Input [1]: [d_date_sk#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#87] -Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:5 Hosting operator id = 62 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#55 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:7 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] -Subquery:8 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] +Subquery:8 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] 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 1cdf12e0cc261..7c5b18e56fa5f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (27) + WholeStageCodegen (29) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (26) + WholeStageCodegen (28) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (18) Project [cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] @@ -81,19 +81,19 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (15) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (7) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #10 - 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] + WholeStageCodegen (16) + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #10 + WholeStageCodegen (7) + HashAggregate [csales] [max,max] + InputAdapter + SortAggregate [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] + WholeStageCodegen (6) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter @@ -123,8 +123,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 - 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)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + InputAdapter + SortAggregate [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)),ssales,sum,isEmpty] + WholeStageCodegen (15) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter @@ -151,72 +152,72 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (27) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (19) + WholeStageCodegen (20) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #13 - WholeStageCodegen (18) + WholeStageCodegen (19) Filter [c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - WholeStageCodegen (24) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + WholeStageCodegen (26) + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + InputAdapter + SortAggregate [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)),ssales,sum,isEmpty] + WholeStageCodegen (25) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (21) + WholeStageCodegen (22) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 - WholeStageCodegen (54) + WholeStageCodegen (58) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #14 - WholeStageCodegen (53) + WholeStageCodegen (57) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (44) + WholeStageCodegen (47) Project [ws_bill_customer_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (37) + WholeStageCodegen (39) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (36) + WholeStageCodegen (38) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_item_sk,item_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (31) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #16 - WholeStageCodegen (28) + WholeStageCodegen (30) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (35) + WholeStageCodegen (37) Sort [item_sk] Project [item_sk] Filter [cnt] @@ -225,62 +226,62 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (32) + WholeStageCodegen (34) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk,d_date] #5 InputAdapter - WholeStageCodegen (34) + WholeStageCodegen (36) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 InputAdapter - WholeStageCodegen (42) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + WholeStageCodegen (45) + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + InputAdapter + SortAggregate [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)),ssales,sum,isEmpty] + WholeStageCodegen (44) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (39) + WholeStageCodegen (41) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - WholeStageCodegen (52) + WholeStageCodegen (56) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (46) + WholeStageCodegen (49) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 InputAdapter - WholeStageCodegen (51) - Sort [c_customer_sk] - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,sum,isEmpty] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + WholeStageCodegen (55) + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + InputAdapter + SortAggregate [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)),ssales,sum,isEmpty] + WholeStageCodegen (54) Project [ss_quantity,ss_sales_price,c_customer_sk] SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (48) + WholeStageCodegen (51) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (50) + WholeStageCodegen (53) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 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 b15ae61d824d4..b8826eb3ec175 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,63 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * HashAggregate (55) - +- * HashAggregate (54) - +- * Project (53) - +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildLeft (41) - : :- BroadcastExchange (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.customer_address (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.store (4) - : +- * HashAggregate (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * SortMergeJoin Inner (37) - : :- * Sort (31) - : : +- Exchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- Union (19) - : : : : :- * Project (14) - : : : : : +- * Filter (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- Scan parquet default.catalog_sales (11) - : : : : +- * Project (18) - : : : : +- * Filter (17) - : : : : +- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (20) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.item (23) - : +- * Sort (36) - : +- Exchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer (32) - +- * Sort (51) - +- Exchange (50) - +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.store_sales (44) - +- ReusedExchange (47) +TakeOrderedAndProject (58) ++- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- SortAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildLeft (41) + : :- BroadcastExchange (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.customer_address (1) + : : +- BroadcastExchange (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet default.store (4) + : +- * HashAggregate (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * SortMergeJoin Inner (37) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- Union (19) + : : : : :- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.catalog_sales (11) + : : : : +- * Project (18) + : : : : +- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (20) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.item (23) + : +- * Sort (36) + : +- Exchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer (32) + +- * Sort (51) + +- Exchange (50) + +- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Filter (46) + : +- * ColumnarToRow (45) + : +- Scan parquet default.store_sales (44) + +- ReusedExchange (47) (1) Scan parquet default.customer_address @@ -145,7 +144,7 @@ Input [3]: [ws_item_sk#15, ws_bill_customer_sk#16, ws_sold_date_sk#17] (19) Union -(20) ReusedExchange [Reuses operator id: 64] +(20) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#21] (21) BroadcastHashJoin [codegen id : 7] @@ -269,7 +268,7 @@ Input [3]: [ss_customer_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] Input [3]: [ss_customer_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] Condition : isnotnull(ss_customer_sk#30) -(47) ReusedExchange [Reuses operator id: 69] +(47) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#34] (48) BroadcastHashJoin [codegen id : 13] @@ -298,196 +297,189 @@ Join condition: None Output [2]: [c_customer_sk#27, ss_ext_sales_price#31] Input [3]: [c_customer_sk#27, ss_customer_sk#30, ss_ext_sales_price#31] -(54) HashAggregate [codegen id : 15] +(54) SortAggregate Input [2]: [c_customer_sk#27, ss_ext_sales_price#31] Keys [1]: [c_customer_sk#27] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [1]: [sum#36] -Results [2]: [c_customer_sk#27, sum#37] - -(55) HashAggregate [codegen id : 15] -Input [2]: [c_customer_sk#27, sum#37] -Keys [1]: [c_customer_sk#27] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#31))#38] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#38,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#31))#36] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#36,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#37] -(56) HashAggregate [codegen id : 15] -Input [1]: [segment#39] -Keys [1]: [segment#39] +(55) HashAggregate [codegen id : 16] +Input [1]: [segment#37] +Keys [1]: [segment#37] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#40] -Results [2]: [segment#39, count#41] +Aggregate Attributes [1]: [count#38] +Results [2]: [segment#37, count#39] -(57) Exchange -Input [2]: [segment#39, count#41] -Arguments: hashpartitioning(segment#39, 5), ENSURE_REQUIREMENTS, [id=#42] +(56) Exchange +Input [2]: [segment#37, count#39] +Arguments: hashpartitioning(segment#37, 5), ENSURE_REQUIREMENTS, [id=#40] -(58) HashAggregate [codegen id : 16] -Input [2]: [segment#39, count#41] -Keys [1]: [segment#39] +(57) HashAggregate [codegen id : 17] +Input [2]: [segment#37, count#39] +Keys [1]: [segment#37] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#43] -Results [3]: [segment#39, count(1)#43 AS num_customers#44, (segment#39 * 50) AS segment_base#45] +Aggregate Attributes [1]: [count(1)#41] +Results [3]: [segment#37, count(1)#41 AS num_customers#42, (segment#37 * 50) AS segment_base#43] -(59) TakeOrderedAndProject -Input [3]: [segment#39, num_customers#44, segment_base#45] -Arguments: 100, [segment#39 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#39, num_customers#44, segment_base#45] +(58) TakeOrderedAndProject +Input [3]: [segment#37, num_customers#42, segment_base#43] +Arguments: 100, [segment#37 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#37, num_customers#42, segment_base#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (64) -+- * Project (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet default.date_dim (60) +BroadcastExchange (63) ++- * Project (62) + +- * Filter (61) + +- * ColumnarToRow (60) + +- Scan parquet default.date_dim (59) -(60) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#46, d_moy#47] +(59) Scan parquet default.date_dim +Output [3]: [d_date_sk#21, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#21, d_year#46, d_moy#47] +(60) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#21, d_year#44, d_moy#45] -(62) Filter [codegen id : 1] -Input [3]: [d_date_sk#21, d_year#46, d_moy#47] -Condition : ((((isnotnull(d_moy#47) AND isnotnull(d_year#46)) AND (d_moy#47 = 12)) AND (d_year#46 = 1998)) AND isnotnull(d_date_sk#21)) +(61) Filter [codegen id : 1] +Input [3]: [d_date_sk#21, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 = 12)) AND (d_year#44 = 1998)) AND isnotnull(d_date_sk#21)) -(63) Project [codegen id : 1] +(62) Project [codegen id : 1] Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#46, d_moy#47] +Input [3]: [d_date_sk#21, d_year#44, d_moy#45] -(64) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#17 IN dynamicpruning#11 Subquery:3 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (69) -+- * Project (68) - +- * Filter (67) - +- * ColumnarToRow (66) - +- Scan parquet default.date_dim (65) +BroadcastExchange (68) ++- * Project (67) + +- * Filter (66) + +- * ColumnarToRow (65) + +- Scan parquet default.date_dim (64) -(65) Scan parquet default.date_dim -Output [2]: [d_date_sk#34, d_month_seq#49] +(64) Scan parquet default.date_dim +Output [2]: [d_date_sk#34, d_month_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_month_seq#49] +(65) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#34, d_month_seq#47] -(67) Filter [codegen id : 1] -Input [2]: [d_date_sk#34, d_month_seq#49] -Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= Subquery scalar-subquery#50, [id=#51])) AND (d_month_seq#49 <= Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#34)) +(66) Filter [codegen id : 1] +Input [2]: [d_date_sk#34, d_month_seq#47] +Condition : (((isnotnull(d_month_seq#47) AND (d_month_seq#47 >= Subquery scalar-subquery#48, [id=#49])) AND (d_month_seq#47 <= Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#34)) -(68) Project [codegen id : 1] +(67) Project [codegen id : 1] Output [1]: [d_date_sk#34] -Input [2]: [d_date_sk#34, d_month_seq#49] +Input [2]: [d_date_sk#34, d_month_seq#47] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -Subquery:4 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* HashAggregate (76) -+- Exchange (75) - +- * HashAggregate (74) - +- * Project (73) - +- * Filter (72) - +- * ColumnarToRow (71) - +- Scan parquet default.date_dim (70) +Subquery:4 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (75) ++- Exchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * Filter (71) + +- * ColumnarToRow (70) + +- Scan parquet default.date_dim (69) -(70) Scan parquet default.date_dim -Output [3]: [d_month_seq#55, d_year#56, d_moy#57] +(69) Scan parquet default.date_dim +Output [3]: [d_month_seq#53, d_year#54, d_moy#55] 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 -(71) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +(70) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#53, d_year#54, d_moy#55] -(72) Filter [codegen id : 1] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] -Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) +(71) Filter [codegen id : 1] +Input [3]: [d_month_seq#53, d_year#54, d_moy#55] +Condition : (((isnotnull(d_year#54) AND isnotnull(d_moy#55)) AND (d_year#54 = 1998)) AND (d_moy#55 = 12)) -(73) Project [codegen id : 1] -Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +(72) Project [codegen id : 1] +Output [1]: [(d_month_seq#53 + 1) AS (d_month_seq + 1)#56] +Input [3]: [d_month_seq#53, d_year#54, d_moy#55] -(74) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#58] -Keys [1]: [(d_month_seq + 1)#58] +(73) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 1)#56] +Keys [1]: [(d_month_seq + 1)#56] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#58] +Results [1]: [(d_month_seq + 1)#56] -(75) Exchange -Input [1]: [(d_month_seq + 1)#58] -Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] +(74) Exchange +Input [1]: [(d_month_seq + 1)#56] +Arguments: hashpartitioning((d_month_seq + 1)#56, 5), ENSURE_REQUIREMENTS, [id=#57] -(76) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#58] -Keys [1]: [(d_month_seq + 1)#58] +(75) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 1)#56] +Keys [1]: [(d_month_seq + 1)#56] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#58] +Results [1]: [(d_month_seq + 1)#56] -Subquery:5 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (83) -+- Exchange (82) - +- * HashAggregate (81) - +- * Project (80) - +- * Filter (79) - +- * ColumnarToRow (78) - +- Scan parquet default.date_dim (77) +Subquery:5 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#50, [id=#51] +* HashAggregate (82) ++- Exchange (81) + +- * HashAggregate (80) + +- * Project (79) + +- * Filter (78) + +- * ColumnarToRow (77) + +- Scan parquet default.date_dim (76) -(77) Scan parquet default.date_dim -Output [3]: [d_month_seq#60, d_year#61, d_moy#62] +(76) Scan parquet default.date_dim +Output [3]: [d_month_seq#58, d_year#59, d_moy#60] 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 -(78) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +(77) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#58, d_year#59, d_moy#60] -(79) Filter [codegen id : 1] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] -Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) +(78) Filter [codegen id : 1] +Input [3]: [d_month_seq#58, d_year#59, d_moy#60] +Condition : (((isnotnull(d_year#59) AND isnotnull(d_moy#60)) AND (d_year#59 = 1998)) AND (d_moy#60 = 12)) -(80) Project [codegen id : 1] -Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +(79) Project [codegen id : 1] +Output [1]: [(d_month_seq#58 + 3) AS (d_month_seq + 3)#61] +Input [3]: [d_month_seq#58, d_year#59, d_moy#60] -(81) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#63] -Keys [1]: [(d_month_seq + 3)#63] +(80) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 3)#61] +Keys [1]: [(d_month_seq + 3)#61] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#63] +Results [1]: [(d_month_seq + 3)#61] -(82) Exchange -Input [1]: [(d_month_seq + 3)#63] -Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] +(81) Exchange +Input [1]: [(d_month_seq + 3)#61] +Arguments: hashpartitioning((d_month_seq + 3)#61, 5), ENSURE_REQUIREMENTS, [id=#62] -(83) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#63] -Keys [1]: [(d_month_seq + 3)#63] +(82) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 3)#61] +Keys [1]: [(d_month_seq + 3)#61] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#63] +Results [1]: [(d_month_seq + 3)#61] 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 77fb803fcf3a5..ce221379977e4 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 @@ -1,135 +1,136 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (16) + WholeStageCodegen (17) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (15) + WholeStageCodegen (16) HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] - InputAdapter - WholeStageCodegen (11) - Sort [c_customer_sk] - Project [c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ca_address_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Filter [ca_address_sk,ca_county,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [s_county,s_state] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_county,s_state] - HashAggregate [c_customer_sk,c_current_addr_sk] + InputAdapter + SortAggregate [c_customer_sk,ss_ext_sales_price] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + WholeStageCodegen (15) + Project [c_customer_sk,ss_ext_sales_price] + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_customer_sk] + Project [c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ca_address_sk] + BroadcastHashJoin [ca_county,ca_state,s_county,s_state] + Filter [ca_address_sk,ca_county,ca_state] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [s_county,s_state] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_county,s_state] HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - SortMergeJoin [customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (8) - Sort [customer_sk] - InputAdapter - Exchange [customer_sk] #4 - WholeStageCodegen (7) - Project [customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - Project [customer_sk,item_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (3) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - 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] - WholeStageCodegen (4) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - 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 - WholeStageCodegen (10) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #7 - WholeStageCodegen (9) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (14) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #8 - WholeStageCodegen (13) - Project [ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + SortMergeJoin [customer_sk,c_customer_sk] InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] + WholeStageCodegen (8) + Sort [customer_sk] + InputAdapter + Exchange [customer_sk] #4 + WholeStageCodegen (7) + Project [customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + Project [customer_sk,item_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] InputAdapter - Exchange [(d_month_seq + 1)] #10 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 1)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] + Union + WholeStageCodegen (3) + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + 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] + WholeStageCodegen (4) + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Exchange [(d_month_seq + 3)] #11 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 3)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + 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 + WholeStageCodegen (10) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #7 + WholeStageCodegen (9) + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #9 + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (14) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #8 + WholeStageCodegen (13) + Project [ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #3 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 1)] + InputAdapter + Exchange [(d_month_seq + 1)] #10 + WholeStageCodegen (1) + HashAggregate [(d_month_seq + 1)] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 3)] + InputAdapter + Exchange [(d_month_seq + 3)] #11 + WholeStageCodegen (1) + HashAggregate [(d_month_seq + 3)] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index da7b19cb7d88b..ddad6bf77e606 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * HashAggregate (41) + +- SortAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] -(41) HashAggregate [codegen id : 11] +(41) SortAggregate Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 13] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 10f43db99224b..842c49b31045d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -1,74 +1,76 @@ -WholeStageCodegen (12) +WholeStageCodegen (13) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] + InputAdapter + SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + WholeStageCodegen (11) + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + Project [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] + ColumnarToRow InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.web_site [web_site_sk,web_company_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] + BroadcastExchange #7 + WholeStageCodegen (10) + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 1c1f76169ca6a..9a80843dd06fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * HashAggregate (41) + +- SortAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] -(41) HashAggregate [codegen id : 11] +(41) SortAggregate Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 11] +(42) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 13] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 481c9e7c3ff4f..7f92108322863 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,74 +1,76 @@ -WholeStageCodegen (12) +WholeStageCodegen (13) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] + InputAdapter + SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + WholeStageCodegen (11) + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + Project [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [d_date_sk] - Filter [d_date,d_date_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #7 + WholeStageCodegen (10) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] + Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 1a24233541a26..c3405bb20427f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -3,7 +3,7 @@ +- Exchange (56) +- * HashAggregate (55) +- * HashAggregate (54) - +- * HashAggregate (53) + +- SortAggregate (53) +- * Project (52) +- * BroadcastHashJoin Inner BuildRight (51) :- * Project (45) @@ -283,21 +283,21 @@ Join condition: None Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#24] -(53) HashAggregate [codegen id : 20] +(53) SortAggregate Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(54) HashAggregate [codegen id : 20] +(54) HashAggregate [codegen id : 21] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(55) HashAggregate [codegen id : 20] +(55) HashAggregate [codegen id : 21] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] @@ -308,7 +308,7 @@ Results [3]: [sum#29, sum#30, count#32] Input [3]: [sum#29, sum#30, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(57) HashAggregate [codegen id : 21] +(57) HashAggregate [codegen id : 22] Input [3]: [sum#29, sum#30, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index c67b17c79364c..4d6c1fc775892 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -1,101 +1,103 @@ -WholeStageCodegen (21) +WholeStageCodegen (22) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (20) + WholeStageCodegen (21) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] + InputAdapter + SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + WholeStageCodegen (20) + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) + Project [wr_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow + WholeStageCodegen (13) + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (10) + Sort [wr_order_number] InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (13) - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (15) - Sort [ws_order_number] + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + Project [wr_order_number] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (15) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (17) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] + ColumnarToRow InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #6 + WholeStageCodegen (18) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.web_site [web_site_sk,web_company_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] + BroadcastExchange #7 + WholeStageCodegen (19) + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 79519bfde99f6..bf278bd4f2ef5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -3,7 +3,7 @@ +- Exchange (57) +- * HashAggregate (56) +- * HashAggregate (55) - +- * HashAggregate (54) + +- SortAggregate (54) +- * Project (53) +- * BroadcastHashJoin Inner BuildRight (52) :- * Project (46) @@ -288,21 +288,21 @@ Join condition: None Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#24] -(54) HashAggregate [codegen id : 20] +(54) SortAggregate Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(55) HashAggregate [codegen id : 20] +(55) HashAggregate [codegen id : 21] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(56) HashAggregate [codegen id : 20] +(56) HashAggregate [codegen id : 21] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] @@ -313,7 +313,7 @@ Results [3]: [sum#29, sum#30, count#32] Input [3]: [sum#29, sum#30, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(58) HashAggregate [codegen id : 21] +(58) HashAggregate [codegen id : 22] Input [3]: [sum#29, sum#30, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 18a866c459e79..ebe5446364fc6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,102 +1,104 @@ -WholeStageCodegen (21) +WholeStageCodegen (22) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (20) + WholeStageCodegen (21) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow + InputAdapter + SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + WholeStageCodegen (20) + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (10) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + Project [wr_order_number] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (15) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] + WholeStageCodegen (12) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow + WholeStageCodegen (14) + Sort [ws_order_number] InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (15) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (14) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (17) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) - Project [d_date_sk] - Filter [d_date,d_date_sk] + BroadcastExchange #6 + WholeStageCodegen (18) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #7 + WholeStageCodegen (19) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] + Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt index 19240a79cc91c..cf42329efa257 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt @@ -68,7 +68,7 @@ : : : : : : : : : : : : : : : : +- * Filter (31) : : : : : : : : : : : : : : : : +- * HashAggregate (30) : : : : : : : : : : : : : : : : +- Exchange (29) - : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- SortAggregate (28) : : : : : : : : : : : : : : : : +- * Project (27) : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (26) : : : : : : : : : : : : : : : : :- * Sort (19) @@ -329,7 +329,7 @@ Join condition: None Output [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Input [8]: [cs_item_sk#19, cs_order_number#20, cs_ext_list_price#21, cr_item_sk#24, cr_order_number#25, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] -(28) HashAggregate [codegen id : 9] +(28) SortAggregate Input [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Keys [1]: [cs_item_sk#19] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#21)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt index b5ebf7af31bed..2c2e97ce2b0ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt @@ -116,8 +116,8 @@ WholeStageCodegen (88) HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #13 - WholeStageCodegen (9) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + SortAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + WholeStageCodegen (9) Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index ddaa34ab4e657..361c3b6751927 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -56,7 +56,7 @@ : : : : : : : : : : : : : : : : +- * Filter (30) : : : : : : : : : : : : : : : : +- * HashAggregate (29) : : : : : : : : : : : : : : : : +- Exchange (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) + : : : : : : : : : : : : : : : : +- SortAggregate (27) : : : : : : : : : : : : : : : : +- * Project (26) : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (25) : : : : : : : : : : : : : : : : :- * Sort (18) @@ -299,7 +299,7 @@ Join condition: None Output [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Input [8]: [cs_item_sk#19, cs_order_number#20, cs_ext_list_price#21, cr_item_sk#24, cr_order_number#25, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] -(27) HashAggregate [codegen id : 8] +(27) SortAggregate Input [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Keys [1]: [cs_item_sk#19] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#21)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 6917f8f6c6e2d..52687c27e0f8a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -80,8 +80,8 @@ WholeStageCodegen (54) HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #6 - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + SortAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + WholeStageCodegen (8) Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter 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 32428fbde0016..9b6dfc78752f2 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 @@ -694,7 +694,8 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils } test("SPARK-25497: LIMIT within whole stage codegen should not consume all the inputs") { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "false") { // A special query that only has one partition, so there is no shuffle and the entire query // can be whole-stage-codegened. val df = spark.range(0, 1500, 1, 1).limit(10).groupBy('id).count().limit(1).filter('id >= 0) From cff1424c07d8423bd9d05c8f001b136dcbb26a75 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Wed, 1 Dec 2021 15:16:13 -0800 Subject: [PATCH 4/5] Check the approach to check partial agg based on logical plan instead --- .../execution/ReplaceHashWithSortAgg.scala | 49 +++---------------- 1 file changed, 8 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala index 77a0681ca9876..63ad2d0cafb75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAgg.scala @@ -93,47 +93,14 @@ object ReplaceHashWithSortAgg extends Rule[SparkPlan] { * Check if `partialAgg` to be partial aggregate of `finalAgg`. */ private def isPartialAgg(partialAgg: HashAggregateExec, finalAgg: HashAggregateExec): Boolean = { - val partialGroupExprs = partialAgg.groupingExpressions - val finalGroupExprs = finalAgg.groupingExpressions - val partialAggExprs = partialAgg.aggregateExpressions - val finalAggExprs = finalAgg.aggregateExpressions - val partialAggAttrs = partialAggExprs.flatMap(_.aggregateFunction.aggBufferAttributes) - val finalAggAttrs = finalAggExprs.map(_.resultAttribute) - val partialResultExprs = partialGroupExprs ++ - partialAggExprs.flatMap(_.aggregateFunction.inputAggBufferAttributes) - - val groupExprsEqual = partialGroupExprs.length == finalGroupExprs.length && - partialGroupExprs.zip(finalGroupExprs).forall { - case (e1, e2) => e1.semanticEquals(e2) - } - val aggExprsEqual = partialAggExprs.length == finalAggExprs.length && - partialAggExprs.forall(_.mode == Partial) && finalAggExprs.forall(_.mode == Final) && - partialAggExprs.zip(finalAggExprs).forall { - case (e1, e2) => e1.aggregateFunction.semanticEquals(e2.aggregateFunction) - } - val isPartialAggAttrsValid = partialAggAttrs.length == partialAgg.aggregateAttributes.length && - partialAggAttrs.zip(partialAgg.aggregateAttributes).forall { - case (a1, a2) => a1.semanticEquals(a2) + if (partialAgg.aggregateExpressions.forall(_.mode == Partial) && + finalAgg.aggregateExpressions.forall(_.mode == Final)) { + (finalAgg.logicalLink, partialAgg.logicalLink) match { + case (Some(agg1), Some(agg2)) => agg1.sameResult(agg2) + case _ => false } - val isFinalAggAttrsValid = finalAggAttrs.length == finalAgg.aggregateAttributes.length && - finalAggAttrs.zip(finalAgg.aggregateAttributes).forall { - case (a1, a2) => a1.semanticEquals(a2) - } - val isPartialResultExprsValid = - partialResultExprs.length == partialAgg.resultExpressions.length && - partialResultExprs.zip(partialAgg.resultExpressions).forall { - case (a1, a2) => a1.semanticEquals(a2) - } - val isRequiredDistributionValid = - partialAgg.requiredChildDistributionExpressions.isEmpty && - finalAgg.requiredChildDistributionExpressions.exists { exprs => - exprs.length == finalGroupExprs.length && - exprs.zip(finalGroupExprs).forall { - case (e1, e2) => e1.semanticEquals(e2) - } - } - - groupExprsEqual && aggExprsEqual && isPartialAggAttrsValid && isFinalAggAttrsValid && - isPartialResultExprsValid && isRequiredDistributionValid + } else { + false + } } } From 8ce7d279cec23d3bc50c8d6f870a10b11395523c Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Thu, 2 Dec 2021 19:38:41 -0800 Subject: [PATCH 5/5] Disable rule by default and back out TPCDS plan change --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../approved-plans-v1_4/q16.sf100/explain.txt | 10 +- .../q16.sf100/simplified.txt | 126 ++- .../approved-plans-v1_4/q16/explain.txt | 10 +- .../approved-plans-v1_4/q16/simplified.txt | 126 ++- .../q23a.sf100/explain.txt | 604 ++++++------ .../q23a.sf100/simplified.txt | 71 +- .../q23b.sf100/explain.txt | 856 ++++++++++-------- .../q23b.sf100/simplified.txt | 121 ++- .../approved-plans-v1_4/q54.sf100/explain.txt | 346 +++---- .../q54.sf100/simplified.txt | 249 +++-- .../approved-plans-v1_4/q94.sf100/explain.txt | 10 +- .../q94.sf100/simplified.txt | 126 ++- .../approved-plans-v1_4/q94/explain.txt | 10 +- .../approved-plans-v1_4/q94/simplified.txt | 126 ++- .../approved-plans-v1_4/q95.sf100/explain.txt | 10 +- .../q95.sf100/simplified.txt | 176 ++-- .../approved-plans-v1_4/q95/explain.txt | 10 +- .../approved-plans-v1_4/q95/simplified.txt | 176 ++-- .../approved-plans-v2_7/q64.sf100/explain.txt | 4 +- .../q64.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q64/explain.txt | 4 +- .../approved-plans-v2_7/q64/simplified.txt | 4 +- .../execution/metric/SQLMetricsSuite.scala | 3 +- 24 files changed, 1635 insertions(+), 1549 deletions(-) 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 6e8ad266e4d74..e0a140bff0a32 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 @@ -1509,7 +1509,7 @@ object SQLConf { .doc("Whether to replace hash aggregate node with sort aggregate based on children's ordering") .version("3.3.0") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val STATE_STORE_PROVIDER_CLASS = buildConf("spark.sql.streaming.stateStore.providerClass") diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 008ebc4c7ea98..cd26a5c2495dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- SortAggregate (41) + +- * HashAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#23] -(41) SortAggregate +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 12] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index 4c4cd8d785925..7af14cd98449e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -1,76 +1,74 @@ -WholeStageCodegen (13) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - InputAdapter - SortAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - WholeStageCodegen (11) - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number] InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + Project [cs_warehouse_sk,cs_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [cr_order_number] + InputAdapter + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + Project [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [cc_call_center_sk] - Filter [cc_county,cc_call_center_sk] + BroadcastExchange #5 + WholeStageCodegen (8) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_county] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [d_date_sk] - Filter [d_date,d_date_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [cc_call_center_sk] + Filter [cc_county,cc_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.call_center [cc_call_center_sk,cc_county] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 4279f75e0a84d..210be116962bb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- SortAggregate (41) + +- * HashAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#23] -(41) SortAggregate +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 12] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] Results [3]: [cs_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index d4624703912a7..126e03fa31d76 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,76 +1,74 @@ -WholeStageCodegen (13) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - InputAdapter - SortAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - WholeStageCodegen (11) - Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] - Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - SortMergeJoin [cs_order_number,cr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #2 - WholeStageCodegen (1) - Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [cs_order_number] - InputAdapter - Exchange [cs_order_number] #3 - WholeStageCodegen (3) - Project [cs_warehouse_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [cr_order_number] - InputAdapter - Exchange [cr_order_number] #4 - WholeStageCodegen (6) - Project [cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + Project [cs_warehouse_sk,cs_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [cr_order_number] + InputAdapter + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + Project [cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #5 + WholeStageCodegen (8) + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [cc_call_center_sk] - Filter [cc_county,cc_call_center_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_county] + Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + Project [cc_call_center_sk] + Filter [cc_county,cc_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.call_center [cc_call_center_sk,cc_county] 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 ba2475adc2ae6..be706fee66776 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,12 +1,12 @@ == Physical Plan == -* HashAggregate (86) -+- Exchange (85) - +- * HashAggregate (84) - +- Union (83) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (46) - : : +- * SortMergeJoin LeftSemi (45) +* HashAggregate (90) ++- Exchange (89) + +- * HashAggregate (88) + +- Union (87) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (48) + : : +- * SortMergeJoin LeftSemi (47) : : :- * Sort (28) : : : +- Exchange (27) : : : +- * Project (26) @@ -35,56 +35,60 @@ : : : +- * Filter (15) : : : +- * ColumnarToRow (14) : : : +- Scan parquet default.item (13) - : : +- * Project (44) - : : +- * Filter (43) - : : +- SortAggregate (42) - : : +- * Project (41) - : : +- * SortMergeJoin Inner (40) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * Filter (31) - : : : +- * ColumnarToRow (30) - : : : +- Scan parquet default.store_sales (29) - : : +- * Sort (39) - : : +- Exchange (38) - : : +- * Filter (37) - : : +- * ColumnarToRow (36) - : : +- Scan parquet default.customer (35) - : +- ReusedExchange (47) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * SortMergeJoin LeftSemi (78) - : :- * Sort (68) - : : +- Exchange (67) - : : +- * Project (66) - : : +- * SortMergeJoin LeftSemi (65) - : : :- * Sort (53) - : : : +- Exchange (52) - : : : +- * ColumnarToRow (51) - : : : +- Scan parquet default.web_sales (50) - : : +- * Sort (64) - : : +- * Project (63) - : : +- * Filter (62) - : : +- * HashAggregate (61) - : : +- * HashAggregate (60) - : : +- * Project (59) - : : +- * SortMergeJoin Inner (58) - : : :- * Sort (55) - : : : +- ReusedExchange (54) - : : +- * Sort (57) - : : +- ReusedExchange (56) - : +- * Project (77) - : +- * Filter (76) - : +- SortAggregate (75) - : +- * Project (74) - : +- * SortMergeJoin Inner (73) - : :- * Sort (70) - : : +- ReusedExchange (69) - : +- * Sort (72) - : +- ReusedExchange (71) - +- ReusedExchange (80) + : : +- * Sort (46) + : : +- * Project (45) + : : +- * Filter (44) + : : +- * HashAggregate (43) + : : +- * HashAggregate (42) + : : +- * Project (41) + : : +- * SortMergeJoin Inner (40) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * Filter (31) + : : : +- * ColumnarToRow (30) + : : : +- Scan parquet default.store_sales (29) + : : +- * Sort (39) + : : +- Exchange (38) + : : +- * Filter (37) + : : +- * ColumnarToRow (36) + : : +- Scan parquet default.customer (35) + : +- ReusedExchange (49) + +- * Project (86) + +- * BroadcastHashJoin Inner BuildRight (85) + :- * Project (83) + : +- * SortMergeJoin LeftSemi (82) + : :- * Sort (70) + : : +- Exchange (69) + : : +- * Project (68) + : : +- * SortMergeJoin LeftSemi (67) + : : :- * Sort (55) + : : : +- Exchange (54) + : : : +- * ColumnarToRow (53) + : : : +- Scan parquet default.web_sales (52) + : : +- * Sort (66) + : : +- * Project (65) + : : +- * Filter (64) + : : +- * HashAggregate (63) + : : +- * HashAggregate (62) + : : +- * Project (61) + : : +- * SortMergeJoin Inner (60) + : : :- * Sort (57) + : : : +- ReusedExchange (56) + : : +- * Sort (59) + : : +- ReusedExchange (58) + : +- * Sort (81) + : +- * Project (80) + : +- * Filter (79) + : +- * HashAggregate (78) + : +- * HashAggregate (77) + : +- * Project (76) + : +- * SortMergeJoin Inner (75) + : :- * Sort (72) + : : +- ReusedExchange (71) + : +- * Sort (74) + : +- ReusedExchange (73) + +- ReusedExchange (84) (1) Scan parquet default.catalog_sales @@ -120,7 +124,7 @@ Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(8) ReusedExchange [Reuses operator id: 96] +(8) ReusedExchange [Reuses operator id: 100] Output [2]: [d_date_sk#11, d_date#12] (9) BroadcastHashJoin [codegen id : 4] @@ -271,394 +275,424 @@ Join condition: None Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(42) SortAggregate +(42) HashAggregate [codegen id : 15] Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] +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#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] + +(43) HashAggregate [codegen id : 15] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#31] -Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] +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))#35] +Results [2]: [c_customer_sk#29, 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 ssales#36] -(43) Filter [codegen id : 16] -Input [2]: [c_customer_sk#29, ssales#32] -Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) +(44) Filter [codegen id : 15] +Input [2]: [c_customer_sk#29, ssales#36] +Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) -(44) Project [codegen id : 16] +(45) Project [codegen id : 15] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#32] +Input [2]: [c_customer_sk#29, ssales#36] + +(46) Sort [codegen id : 15] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(45) SortMergeJoin [codegen id : 18] +(47) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(46) Project [codegen id : 18] +(48) Project [codegen id : 17] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(47) ReusedExchange [Reuses operator id: 91] -Output [1]: [d_date_sk#35] +(49) ReusedExchange [Reuses operator id: 95] +Output [1]: [d_date_sk#39] -(48) BroadcastHashJoin [codegen id : 18] +(50) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#35] +Right keys [1]: [d_date_sk#39] Join condition: None -(49) Project [codegen id : 18] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#36] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#35] +(51) Project [codegen id : 17] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#40] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#39] -(50) Scan parquet default.web_sales -Output [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +(52) Scan parquet default.web_sales +Output [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 19] -Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +(53) ColumnarToRow [codegen id : 18] +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -(52) Exchange -Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -Arguments: hashpartitioning(ws_item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#42] +(54) Exchange +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Arguments: hashpartitioning(ws_item_sk#41, 5), ENSURE_REQUIREMENTS, [id=#46] -(53) Sort [codegen id : 20] -Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -Arguments: [ws_item_sk#37 ASC NULLS FIRST], false, 0 +(55) Sort [codegen id : 19] +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Arguments: [ws_item_sk#41 ASC NULLS FIRST], false, 0 -(54) ReusedExchange [Reuses operator id: 11] +(56) ReusedExchange [Reuses operator id: 11] Output [2]: [ss_item_sk#8, d_date#12] -(55) Sort [codegen id : 23] +(57) Sort [codegen id : 22] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(56) ReusedExchange [Reuses operator id: 16] +(58) ReusedExchange [Reuses operator id: 16] Output [2]: [i_item_sk#14, i_item_desc#15] -(57) Sort [codegen id : 25] +(59) Sort [codegen id : 24] Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 26] +(60) SortMergeJoin [codegen id : 25] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#14] Join condition: None -(59) Project [codegen id : 26] -Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#43] +(61) Project [codegen id : 25] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#47] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] -(60) HashAggregate [codegen id : 26] -Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#43] -Keys [3]: [_groupingexpression#43, i_item_sk#14, d_date#12] +(62) HashAggregate [codegen id : 25] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#47] +Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] -Results [4]: [_groupingexpression#43, i_item_sk#14, d_date#12, count#19] +Results [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] -(61) HashAggregate [codegen id : 26] -Input [4]: [_groupingexpression#43, i_item_sk#14, d_date#12, count#19] -Keys [3]: [_groupingexpression#43, i_item_sk#14, d_date#12] +(63) HashAggregate [codegen id : 25] +Input [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] -(62) Filter [codegen id : 26] +(64) Filter [codegen id : 25] Input [2]: [item_sk#21, cnt#22] Condition : (cnt#22 > 4) -(63) Project [codegen id : 26] +(65) Project [codegen id : 25] Output [1]: [item_sk#21] Input [2]: [item_sk#21, cnt#22] -(64) Sort [codegen id : 26] +(66) Sort [codegen id : 25] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 27] -Left keys [1]: [ws_item_sk#37] +(67) SortMergeJoin [codegen id : 26] +Left keys [1]: [ws_item_sk#41] Right keys [1]: [item_sk#21] Join condition: None -(66) Project [codegen id : 27] -Output [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -Input [5]: [ws_item_sk#37, ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +(68) Project [codegen id : 26] +Output [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -(67) Exchange -Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -Arguments: hashpartitioning(ws_bill_customer_sk#38, 5), ENSURE_REQUIREMENTS, [id=#44] +(69) Exchange +Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#48] -(68) Sort [codegen id : 28] -Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#38 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 27] +Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 -(69) ReusedExchange [Reuses operator id: 33] +(71) ReusedExchange [Reuses operator id: 33] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(70) Sort [codegen id : 30] +(72) 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 -(71) ReusedExchange [Reuses operator id: 38] +(73) ReusedExchange [Reuses operator id: 38] Output [1]: [c_customer_sk#29] -(72) Sort [codegen id : 32] +(74) Sort [codegen id : 31] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(73) SortMergeJoin [codegen id : 33] +(75) SortMergeJoin [codegen id : 32] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(74) Project [codegen id : 33] +(76) Project [codegen id : 32] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(75) SortAggregate +(77) HashAggregate [codegen id : 32] Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] +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#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] + +(78) HashAggregate [codegen id : 32] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#31] -Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] +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))#35] +Results [2]: [c_customer_sk#29, 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 ssales#36] -(76) Filter [codegen id : 34] -Input [2]: [c_customer_sk#29, ssales#32] -Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) +(79) Filter [codegen id : 32] +Input [2]: [c_customer_sk#29, ssales#36] +Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) -(77) Project [codegen id : 34] +(80) Project [codegen id : 32] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#32] +Input [2]: [c_customer_sk#29, ssales#36] + +(81) Sort [codegen id : 32] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(78) SortMergeJoin [codegen id : 36] -Left keys [1]: [ws_bill_customer_sk#38] +(82) SortMergeJoin [codegen id : 34] +Left keys [1]: [ws_bill_customer_sk#42] Right keys [1]: [c_customer_sk#29] Join condition: None -(79) Project [codegen id : 36] -Output [3]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] -Input [4]: [ws_bill_customer_sk#38, ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41] +(83) Project [codegen id : 34] +Output [3]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -(80) ReusedExchange [Reuses operator id: 91] -Output [1]: [d_date_sk#45] +(84) ReusedExchange [Reuses operator id: 95] +Output [1]: [d_date_sk#49] -(81) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ws_sold_date_sk#41] -Right keys [1]: [d_date_sk#45] +(85) BroadcastHashJoin [codegen id : 34] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#49] Join condition: None -(82) Project [codegen id : 36] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#39 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#40 as decimal(12,2)))), DecimalType(18,2), true) AS sales#46] -Input [4]: [ws_quantity#39, ws_list_price#40, ws_sold_date_sk#41, d_date_sk#45] +(86) Project [codegen id : 34] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#50] +Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#49] -(83) Union +(87) Union -(84) HashAggregate [codegen id : 37] -Input [1]: [sales#36] +(88) HashAggregate [codegen id : 35] +Input [1]: [sales#40] Keys: [] -Functions [1]: [partial_sum(sales#36)] -Aggregate Attributes [2]: [sum#47, isEmpty#48] -Results [2]: [sum#49, isEmpty#50] +Functions [1]: [partial_sum(sales#40)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [2]: [sum#53, isEmpty#54] -(85) Exchange -Input [2]: [sum#49, isEmpty#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] +(89) Exchange +Input [2]: [sum#53, isEmpty#54] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#55] -(86) HashAggregate [codegen id : 38] -Input [2]: [sum#49, isEmpty#50] +(90) HashAggregate [codegen id : 36] +Input [2]: [sum#53, isEmpty#54] Keys: [] -Functions [1]: [sum(sales#36)] -Aggregate Attributes [1]: [sum(sales#36)#52] -Results [1]: [sum(sales#36)#52 AS sum(sales)#53] +Functions [1]: [sum(sales#40)] +Aggregate Attributes [1]: [sum(sales#40)#56] +Results [1]: [sum(sales#40)#56 AS sum(sales)#57] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (91) -+- * Project (90) - +- * Filter (89) - +- * ColumnarToRow (88) - +- Scan parquet default.date_dim (87) +BroadcastExchange (95) ++- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.date_dim (91) -(87) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#54, d_moy#55] +(91) Scan parquet default.date_dim +Output [3]: [d_date_sk#39, d_year#58, d_moy#59] 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 -(88) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#54, d_moy#55] +(92) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] -(89) Filter [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#54, d_moy#55] -Condition : ((((isnotnull(d_year#54) AND isnotnull(d_moy#55)) AND (d_year#54 = 2000)) AND (d_moy#55 = 2)) AND isnotnull(d_date_sk#35)) +(93) Filter [codegen id : 1] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] +Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#39)) -(90) Project [codegen id : 1] -Output [1]: [d_date_sk#35] -Input [3]: [d_date_sk#35, d_year#54, d_moy#55] +(94) Project [codegen id : 1] +Output [1]: [d_date_sk#39] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] -(91) BroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +(95) BroadcastExchange +Input [1]: [d_date_sk#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (96) -+- * Project (95) - +- * Filter (94) - +- * ColumnarToRow (93) - +- Scan parquet default.date_dim (92) +BroadcastExchange (100) ++- * Project (99) + +- * Filter (98) + +- * ColumnarToRow (97) + +- Scan parquet default.date_dim (96) -(92) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#57] +(96) Scan parquet default.date_dim +Output [3]: [d_date_sk#11, d_date#12, d_year#61] 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 -(93) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#57] +(97) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] -(94) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#57] -Condition : (d_year#57 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +(98) Filter [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] +Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(95) Project [codegen id : 1] +(99) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#57] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] -(96) BroadcastExchange +(100) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] - -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#33, [id=#34] -* HashAggregate (112) -+- Exchange (111) - +- * HashAggregate (110) - +- SortAggregate (109) - +- * Project (108) - +- * SortMergeJoin Inner (107) - :- * Sort (104) - : +- Exchange (103) - : +- * Project (102) - : +- * BroadcastHashJoin Inner BuildRight (101) - : :- * Filter (99) - : : +- * ColumnarToRow (98) - : : +- Scan parquet default.store_sales (97) - : +- ReusedExchange (100) - +- * Sort (106) - +- ReusedExchange (105) - - -(97) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] + +Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#37, [id=#38] +* HashAggregate (117) ++- Exchange (116) + +- * HashAggregate (115) + +- * HashAggregate (114) + +- * HashAggregate (113) + +- * Project (112) + +- * SortMergeJoin Inner (111) + :- * Sort (108) + : +- Exchange (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Filter (103) + : : +- * ColumnarToRow (102) + : : +- Scan parquet default.store_sales (101) + : +- ReusedExchange (104) + +- * Sort (110) + +- ReusedExchange (109) + + +(101) Scan parquet default.store_sales +Output [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(98) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +(102) ColumnarToRow [codegen id : 2] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] -(99) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] -Condition : isnotnull(ss_customer_sk#59) +(103) Filter [codegen id : 2] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_customer_sk#63) -(100) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#64] +(104) ReusedExchange [Reuses operator id: 122] +Output [1]: [d_date_sk#68] -(101) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#62] -Right keys [1]: [d_date_sk#64] +(105) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#68] Join condition: None -(102) Project [codegen id : 2] -Output [3]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61] -Input [5]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, d_date_sk#64] +(106) Project [codegen id : 2] +Output [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Input [5]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, d_date_sk#68] -(103) Exchange -Input [3]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61] -Arguments: hashpartitioning(ss_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +(107) Exchange +Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Arguments: hashpartitioning(ss_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] -(104) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61] -Arguments: [ss_customer_sk#59 ASC NULLS FIRST], false, 0 +(108) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Arguments: [ss_customer_sk#63 ASC NULLS FIRST], false, 0 -(105) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#66] +(109) ReusedExchange [Reuses operator id: 38] +Output [1]: [c_customer_sk#70] -(106) Sort [codegen id : 5] -Input [1]: [c_customer_sk#66] -Arguments: [c_customer_sk#66 ASC NULLS FIRST], false, 0 +(110) Sort [codegen id : 5] +Input [1]: [c_customer_sk#70] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 -(107) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#59] -Right keys [1]: [c_customer_sk#66] +(111) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#63] +Right keys [1]: [c_customer_sk#70] Join condition: None -(108) Project [codegen id : 6] -Output [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#66] -Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, c_customer_sk#66] - -(109) SortAggregate -Input [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#66] -Keys [1]: [c_customer_sk#66] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#61 as decimal(12,2)))), DecimalType(18,2), true))#67] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#61 as decimal(12,2)))), DecimalType(18,2), true))#67 AS csales#68] - -(110) HashAggregate [codegen id : 7] -Input [1]: [csales#68] +(112) Project [codegen id : 6] +Output [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, c_customer_sk#70] + +(113) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] +Keys [1]: [c_customer_sk#70] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#71, isEmpty#72] +Results [3]: [c_customer_sk#70, sum#73, isEmpty#74] + +(114) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#70, sum#73, isEmpty#74] +Keys [1]: [c_customer_sk#70] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75 AS csales#76] + +(115) HashAggregate [codegen id : 6] +Input [1]: [csales#76] Keys: [] -Functions [1]: [partial_max(csales#68)] -Aggregate Attributes [1]: [max#69] -Results [1]: [max#70] +Functions [1]: [partial_max(csales#76)] +Aggregate Attributes [1]: [max#77] +Results [1]: [max#78] -(111) Exchange -Input [1]: [max#70] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#71] +(116) Exchange +Input [1]: [max#78] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#79] -(112) HashAggregate [codegen id : 8] -Input [1]: [max#70] +(117) HashAggregate [codegen id : 7] +Input [1]: [max#78] Keys: [] -Functions [1]: [max(csales#68)] -Aggregate Attributes [1]: [max(csales#68)#72] -Results [1]: [max(csales#68)#72 AS tpcds_cmax#73] +Functions [1]: [max(csales#76)] +Aggregate Attributes [1]: [max(csales#76)#80] +Results [1]: [max(csales#76)#80 AS tpcds_cmax#81] -Subquery:4 Hosting operator id = 97 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 -BroadcastExchange (117) -+- * Project (116) - +- * Filter (115) - +- * ColumnarToRow (114) - +- Scan parquet default.date_dim (113) +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +BroadcastExchange (122) ++- * Project (121) + +- * Filter (120) + +- * ColumnarToRow (119) + +- Scan parquet default.date_dim (118) -(113) Scan parquet default.date_dim -Output [2]: [d_date_sk#64, d_year#74] +(118) Scan parquet default.date_dim +Output [2]: [d_date_sk#68, d_year#82] 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 -(114) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#64, d_year#74] +(119) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#68, d_year#82] -(115) Filter [codegen id : 1] -Input [2]: [d_date_sk#64, d_year#74] -Condition : (d_year#74 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) +(120) Filter [codegen id : 1] +Input [2]: [d_date_sk#68, d_year#82] +Condition : (d_year#82 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) -(116) Project [codegen id : 1] -Output [1]: [d_date_sk#64] -Input [2]: [d_date_sk#64, d_year#74] +(121) Project [codegen id : 1] +Output [1]: [d_date_sk#68] +Input [2]: [d_date_sk#68, d_year#82] -(117) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +(122) BroadcastExchange +Input [1]: [d_date_sk#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 76 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] +Subquery:6 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] 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 5c2a89277801a..17377b91326fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -1,12 +1,12 @@ -WholeStageCodegen (38) +WholeStageCodegen (36) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 - WholeStageCodegen (37) + WholeStageCodegen (35) HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (18) + WholeStageCodegen (17) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk] @@ -78,19 +78,19 @@ WholeStageCodegen (38) InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (16) - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #10 - WholeStageCodegen (7) - HashAggregate [csales] [max,max] - InputAdapter - SortAggregate [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] - WholeStageCodegen (6) + WholeStageCodegen (15) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (7) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #10 + 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] InputAdapter @@ -120,9 +120,8 @@ WholeStageCodegen (38) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 - InputAdapter - SortAggregate [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)),ssales,sum,isEmpty] - WholeStageCodegen (15) + 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)),ssales,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 @@ -148,31 +147,31 @@ WholeStageCodegen (38) Scan parquet default.customer [c_customer_sk] InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (36) + WholeStageCodegen (34) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (28) + WholeStageCodegen (27) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #13 - WholeStageCodegen (27) + WholeStageCodegen (26) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_item_sk,item_sk] InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (19) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #14 - WholeStageCodegen (19) + WholeStageCodegen (18) ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (26) + WholeStageCodegen (25) Sort [item_sk] Project [item_sk] Filter [cnt] @@ -181,32 +180,32 @@ WholeStageCodegen (38) Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (22) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk,d_date] #5 InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 InputAdapter - WholeStageCodegen (34) - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - InputAdapter - SortAggregate [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)),ssales,sum,isEmpty] - WholeStageCodegen (33) + WholeStageCodegen (32) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,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 (30) + WholeStageCodegen (29) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (32) + WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 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 735457aa23567..638f5ec3ded62 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,14 +1,14 @@ == Physical Plan == -TakeOrderedAndProject (121) -+- Union (120) - :- * HashAggregate (69) - : +- Exchange (68) - : +- * HashAggregate (67) - : +- * Project (66) - : +- * SortMergeJoin Inner (65) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * SortMergeJoin LeftSemi (46) +TakeOrderedAndProject (129) ++- Union (128) + :- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * Project (70) + : +- * SortMergeJoin Inner (69) + : :- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * SortMergeJoin LeftSemi (48) : : : :- * Sort (29) : : : : +- Exchange (28) : : : : +- * Project (27) @@ -38,88 +38,96 @@ TakeOrderedAndProject (121) : : : : +- * Filter (16) : : : : +- * ColumnarToRow (15) : : : : +- Scan parquet default.item (14) - : : : +- * Project (45) - : : : +- * Filter (44) - : : : +- SortAggregate (43) - : : : +- * Project (42) - : : : +- * SortMergeJoin Inner (41) - : : : :- * Sort (35) - : : : : +- Exchange (34) - : : : : +- * Project (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.store_sales (30) - : : : +- * Sort (40) - : : : +- Exchange (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.customer (36) - : : +- ReusedExchange (47) - : +- * SortMergeJoin LeftSemi (64) - : :- * Sort (54) - : : +- Exchange (53) - : : +- * Filter (52) - : : +- * ColumnarToRow (51) - : : +- Scan parquet default.customer (50) - : +- * Project (63) - : +- * Filter (62) - : +- SortAggregate (61) - : +- * Project (60) - : +- * SortMergeJoin Inner (59) - : :- * Sort (56) - : : +- ReusedExchange (55) - : +- * Sort (58) - : +- ReusedExchange (57) - +- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- * Project (116) - +- * SortMergeJoin Inner (115) - :- * Project (102) - : +- * BroadcastHashJoin Inner BuildRight (101) - : :- * SortMergeJoin LeftSemi (99) - : : :- * Sort (89) - : : : +- Exchange (88) - : : : +- * Project (87) - : : : +- * SortMergeJoin LeftSemi (86) - : : : :- * Sort (74) - : : : : +- Exchange (73) - : : : : +- * Filter (72) - : : : : +- * ColumnarToRow (71) - : : : : +- Scan parquet default.web_sales (70) - : : : +- * Sort (85) - : : : +- * Project (84) - : : : +- * Filter (83) - : : : +- * HashAggregate (82) - : : : +- * HashAggregate (81) - : : : +- * Project (80) - : : : +- * SortMergeJoin Inner (79) - : : : :- * Sort (76) - : : : : +- ReusedExchange (75) - : : : +- * Sort (78) - : : : +- ReusedExchange (77) - : : +- * Project (98) - : : +- * Filter (97) - : : +- SortAggregate (96) - : : +- * Project (95) - : : +- * SortMergeJoin Inner (94) - : : :- * Sort (91) - : : : +- ReusedExchange (90) - : : +- * Sort (93) - : : +- ReusedExchange (92) - : +- ReusedExchange (100) - +- * SortMergeJoin LeftSemi (114) - :- * Sort (104) - : +- ReusedExchange (103) - +- * Project (113) - +- * Filter (112) - +- SortAggregate (111) - +- * Project (110) - +- * SortMergeJoin Inner (109) - :- * Sort (106) - : +- ReusedExchange (105) - +- * Sort (108) - +- ReusedExchange (107) + : : : +- * Sort (47) + : : : +- * Project (46) + : : : +- * Filter (45) + : : : +- * HashAggregate (44) + : : : +- * HashAggregate (43) + : : : +- * Project (42) + : : : +- * SortMergeJoin Inner (41) + : : : :- * Sort (35) + : : : : +- Exchange (34) + : : : : +- * Project (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.store_sales (30) + : : : +- * Sort (40) + : : : +- Exchange (39) + : : : +- * Filter (38) + : : : +- * ColumnarToRow (37) + : : : +- Scan parquet default.customer (36) + : : +- ReusedExchange (49) + : +- * SortMergeJoin LeftSemi (68) + : :- * Sort (56) + : : +- Exchange (55) + : : +- * Filter (54) + : : +- * ColumnarToRow (53) + : : +- Scan parquet default.customer (52) + : +- * Sort (67) + : +- * Project (66) + : +- * Filter (65) + : +- * HashAggregate (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * SortMergeJoin Inner (61) + : :- * Sort (58) + : : +- ReusedExchange (57) + : +- * Sort (60) + : +- ReusedExchange (59) + +- * HashAggregate (127) + +- Exchange (126) + +- * HashAggregate (125) + +- * Project (124) + +- * SortMergeJoin Inner (123) + :- * Project (108) + : +- * BroadcastHashJoin Inner BuildRight (107) + : :- * SortMergeJoin LeftSemi (105) + : : :- * Sort (93) + : : : +- Exchange (92) + : : : +- * Project (91) + : : : +- * SortMergeJoin LeftSemi (90) + : : : :- * Sort (78) + : : : : +- Exchange (77) + : : : : +- * Filter (76) + : : : : +- * ColumnarToRow (75) + : : : : +- Scan parquet default.web_sales (74) + : : : +- * Sort (89) + : : : +- * Project (88) + : : : +- * Filter (87) + : : : +- * HashAggregate (86) + : : : +- * HashAggregate (85) + : : : +- * Project (84) + : : : +- * SortMergeJoin Inner (83) + : : : :- * Sort (80) + : : : : +- ReusedExchange (79) + : : : +- * Sort (82) + : : : +- ReusedExchange (81) + : : +- * Sort (104) + : : +- * Project (103) + : : +- * Filter (102) + : : +- * HashAggregate (101) + : : +- * HashAggregate (100) + : : +- * Project (99) + : : +- * SortMergeJoin Inner (98) + : : :- * Sort (95) + : : : +- ReusedExchange (94) + : : +- * Sort (97) + : : +- ReusedExchange (96) + : +- ReusedExchange (106) + +- * SortMergeJoin LeftSemi (122) + :- * Sort (110) + : +- ReusedExchange (109) + +- * Sort (121) + +- * Project (120) + +- * Filter (119) + +- * HashAggregate (118) + +- * HashAggregate (117) + +- * Project (116) + +- * SortMergeJoin Inner (115) + :- * Sort (112) + : +- ReusedExchange (111) + +- * Sort (114) + +- ReusedExchange (113) (1) Scan parquet default.catalog_sales @@ -160,7 +168,7 @@ Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Input [2]: [ss_item_sk#8, ss_sold_date_sk#9] Condition : isnotnull(ss_item_sk#8) -(9) ReusedExchange [Reuses operator id: 131] +(9) ReusedExchange [Reuses operator id: 139] Output [2]: [d_date_sk#11, d_date#12] (10) BroadcastHashJoin [codegen id : 4] @@ -311,550 +319,602 @@ Join condition: None Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(43) SortAggregate +(43) HashAggregate [codegen id : 15] Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] +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#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] + +(44) HashAggregate [codegen id : 15] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#31] -Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] +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))#35] +Results [2]: [c_customer_sk#29, 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 ssales#36] -(44) Filter [codegen id : 16] -Input [2]: [c_customer_sk#29, ssales#32] -Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) +(45) Filter [codegen id : 15] +Input [2]: [c_customer_sk#29, ssales#36] +Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) -(45) Project [codegen id : 16] +(46) Project [codegen id : 15] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#32] +Input [2]: [c_customer_sk#29, ssales#36] + +(47) Sort [codegen id : 15] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(46) SortMergeJoin [codegen id : 18] +(48) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#29] Join condition: None -(47) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#35] +(49) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#39] -(48) BroadcastHashJoin [codegen id : 18] +(50) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#35] +Right keys [1]: [d_date_sk#39] Join condition: None -(49) Project [codegen id : 18] +(51) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#35] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#39] -(50) Scan parquet default.customer -Output [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] +(52) Scan parquet default.customer +Output [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 19] -Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] +(53) ColumnarToRow [codegen id : 18] +Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] -(52) Filter [codegen id : 19] -Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] -Condition : isnotnull(c_customer_sk#36) +(54) Filter [codegen id : 18] +Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] +Condition : isnotnull(c_customer_sk#40) -(53) Exchange -Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] -Arguments: hashpartitioning(c_customer_sk#36, 5), ENSURE_REQUIREMENTS, [id=#39] +(55) Exchange +Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] +Arguments: hashpartitioning(c_customer_sk#40, 5), ENSURE_REQUIREMENTS, [id=#43] -(54) Sort [codegen id : 20] -Input [3]: [c_customer_sk#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#36 ASC NULLS FIRST], false, 0 +(56) Sort [codegen id : 19] +Input [3]: [c_customer_sk#40, c_first_name#41, c_last_name#42] +Arguments: [c_customer_sk#40 ASC NULLS FIRST], false, 0 -(55) ReusedExchange [Reuses operator id: 34] +(57) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(56) Sort [codegen id : 22] +(58) 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 -(57) ReusedExchange [Reuses operator id: 39] +(59) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#29] -(58) Sort [codegen id : 24] +(60) Sort [codegen id : 23] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(59) SortMergeJoin [codegen id : 25] +(61) SortMergeJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(60) Project [codegen id : 25] +(62) Project [codegen id : 24] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(61) SortAggregate +(63) HashAggregate [codegen id : 24] Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] +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#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] + +(64) HashAggregate [codegen id : 24] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#31] -Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] +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))#35] +Results [2]: [c_customer_sk#29, 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 ssales#36] -(62) Filter [codegen id : 26] -Input [2]: [c_customer_sk#29, ssales#32] -Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) +(65) Filter [codegen id : 24] +Input [2]: [c_customer_sk#29, ssales#36] +Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) -(63) Project [codegen id : 26] +(66) Project [codegen id : 24] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#32] +Input [2]: [c_customer_sk#29, ssales#36] + +(67) Sort [codegen id : 24] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin [codegen id : 27] -Left keys [1]: [c_customer_sk#36] +(68) SortMergeJoin [codegen id : 25] +Left keys [1]: [c_customer_sk#40] Right keys [1]: [c_customer_sk#29] Join condition: None -(65) SortMergeJoin [codegen id : 28] +(69) SortMergeJoin [codegen id : 26] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#36] +Right keys [1]: [c_customer_sk#40] Join condition: None -(66) Project [codegen id : 28] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#37, c_last_name#38] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#36, c_first_name#37, c_last_name#38] +(70) Project [codegen id : 26] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#41, c_last_name#42] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#40, c_first_name#41, c_last_name#42] -(67) HashAggregate [codegen id : 28] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#37, c_last_name#38] -Keys [2]: [c_last_name#38, c_first_name#37] +(71) HashAggregate [codegen id : 26] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#41, c_last_name#42] +Keys [2]: [c_last_name#42, c_first_name#41] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [4]: [c_last_name#38, c_first_name#37, sum#42, isEmpty#43] +Aggregate Attributes [2]: [sum#44, isEmpty#45] +Results [4]: [c_last_name#42, c_first_name#41, sum#46, isEmpty#47] -(68) Exchange -Input [4]: [c_last_name#38, c_first_name#37, sum#42, isEmpty#43] -Arguments: hashpartitioning(c_last_name#38, c_first_name#37, 5), ENSURE_REQUIREMENTS, [id=#44] +(72) Exchange +Input [4]: [c_last_name#42, c_first_name#41, sum#46, isEmpty#47] +Arguments: hashpartitioning(c_last_name#42, c_first_name#41, 5), ENSURE_REQUIREMENTS, [id=#48] -(69) HashAggregate [codegen id : 29] -Input [4]: [c_last_name#38, c_first_name#37, sum#42, isEmpty#43] -Keys [2]: [c_last_name#38, c_first_name#37] +(73) HashAggregate [codegen id : 27] +Input [4]: [c_last_name#42, c_first_name#41, sum#46, isEmpty#47] +Keys [2]: [c_last_name#42, c_first_name#41] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45] -Results [3]: [c_last_name#38, c_first_name#37, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#45 AS sales#46] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#49] +Results [3]: [c_last_name#42, c_first_name#41, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#49 AS sales#50] -(70) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +(74) Scan parquet default.web_sales +Output [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#55), dynamicpruningexpression(ws_sold_date_sk#55 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 30] -Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +(75) ColumnarToRow [codegen id : 28] +Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -(72) Filter [codegen id : 30] -Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_customer_sk#48) +(76) Filter [codegen id : 28] +Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +Condition : isnotnull(ws_bill_customer_sk#52) -(73) Exchange -Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -Arguments: hashpartitioning(ws_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#52] +(77) Exchange +Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +Arguments: hashpartitioning(ws_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] -(74) Sort [codegen id : 31] -Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -Arguments: [ws_item_sk#47 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 29] +Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +Arguments: [ws_item_sk#51 ASC NULLS FIRST], false, 0 -(75) ReusedExchange [Reuses operator id: 12] +(79) ReusedExchange [Reuses operator id: 12] Output [2]: [ss_item_sk#8, d_date#12] -(76) Sort [codegen id : 34] +(80) Sort [codegen id : 32] Input [2]: [ss_item_sk#8, d_date#12] Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 17] +(81) ReusedExchange [Reuses operator id: 17] Output [2]: [i_item_sk#14, i_item_desc#15] -(78) Sort [codegen id : 36] +(82) Sort [codegen id : 34] Input [2]: [i_item_sk#14, i_item_desc#15] Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin [codegen id : 37] +(83) SortMergeJoin [codegen id : 35] Left keys [1]: [ss_item_sk#8] Right keys [1]: [i_item_sk#14] Join condition: None -(80) Project [codegen id : 37] -Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#53] +(84) Project [codegen id : 35] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#57] Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] -(81) HashAggregate [codegen id : 37] -Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#53] -Keys [3]: [_groupingexpression#53, i_item_sk#14, d_date#12] +(85) HashAggregate [codegen id : 35] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#57] +Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#18] -Results [4]: [_groupingexpression#53, i_item_sk#14, d_date#12, count#19] +Results [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] -(82) HashAggregate [codegen id : 37] -Input [4]: [_groupingexpression#53, i_item_sk#14, d_date#12, count#19] -Keys [3]: [_groupingexpression#53, i_item_sk#14, d_date#12] +(86) HashAggregate [codegen id : 35] +Input [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] -(83) Filter [codegen id : 37] +(87) Filter [codegen id : 35] Input [2]: [item_sk#21, cnt#22] Condition : (cnt#22 > 4) -(84) Project [codegen id : 37] +(88) Project [codegen id : 35] Output [1]: [item_sk#21] Input [2]: [item_sk#21, cnt#22] -(85) Sort [codegen id : 37] +(89) Sort [codegen id : 35] Input [1]: [item_sk#21] Arguments: [item_sk#21 ASC NULLS FIRST], false, 0 -(86) SortMergeJoin [codegen id : 38] -Left keys [1]: [ws_item_sk#47] +(90) SortMergeJoin [codegen id : 36] +Left keys [1]: [ws_item_sk#51] Right keys [1]: [item_sk#21] Join condition: None -(87) Project [codegen id : 38] -Output [4]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -Input [5]: [ws_item_sk#47, ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] +(91) Project [codegen id : 36] +Output [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -(88) Exchange -Input [4]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -Arguments: hashpartitioning(ws_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] +(92) Exchange +Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +Arguments: hashpartitioning(ws_bill_customer_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] -(89) Sort [codegen id : 39] -Input [4]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51] -Arguments: [ws_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +(93) Sort [codegen id : 37] +Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] +Arguments: [ws_bill_customer_sk#52 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 34] +(94) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(91) Sort [codegen id : 41] +(95) Sort [codegen id : 39] Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 39] +(96) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#29] -(93) Sort [codegen id : 43] +(97) Sort [codegen id : 41] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(94) SortMergeJoin [codegen id : 44] +(98) SortMergeJoin [codegen id : 42] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(95) Project [codegen id : 44] +(99) Project [codegen id : 42] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(96) SortAggregate +(100) HashAggregate [codegen id : 42] Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] +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#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] + +(101) HashAggregate [codegen id : 42] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#31] -Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] +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))#35] +Results [2]: [c_customer_sk#29, 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 ssales#36] -(97) Filter [codegen id : 45] -Input [2]: [c_customer_sk#29, ssales#32] -Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) +(102) Filter [codegen id : 42] +Input [2]: [c_customer_sk#29, ssales#36] +Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) -(98) Project [codegen id : 45] +(103) Project [codegen id : 42] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#32] +Input [2]: [c_customer_sk#29, ssales#36] -(99) SortMergeJoin [codegen id : 47] -Left keys [1]: [ws_bill_customer_sk#48] +(104) Sort [codegen id : 42] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 + +(105) SortMergeJoin [codegen id : 44] +Left keys [1]: [ws_bill_customer_sk#52] Right keys [1]: [c_customer_sk#29] Join condition: None -(100) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#55] +(106) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#59] -(101) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#55] +(107) BroadcastHashJoin [codegen id : 44] +Left keys [1]: [ws_sold_date_sk#55] +Right keys [1]: [d_date_sk#59] Join condition: None -(102) Project [codegen id : 47] -Output [3]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50] -Input [5]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, ws_sold_date_sk#51, d_date_sk#55] +(108) Project [codegen id : 44] +Output [3]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54] +Input [5]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55, d_date_sk#59] -(103) ReusedExchange [Reuses operator id: 53] -Output [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] +(109) ReusedExchange [Reuses operator id: 55] +Output [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] -(104) Sort [codegen id : 49] -Input [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] -Arguments: [c_customer_sk#56 ASC NULLS FIRST], false, 0 +(110) Sort [codegen id : 46] +Input [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] +Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 -(105) ReusedExchange [Reuses operator id: 34] +(111) ReusedExchange [Reuses operator id: 34] Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] -(106) Sort [codegen id : 51] +(112) Sort [codegen id : 48] Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 -(107) ReusedExchange [Reuses operator id: 39] +(113) ReusedExchange [Reuses operator id: 39] Output [1]: [c_customer_sk#29] -(108) Sort [codegen id : 53] +(114) Sort [codegen id : 50] Input [1]: [c_customer_sk#29] Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(109) SortMergeJoin [codegen id : 54] +(115) SortMergeJoin [codegen id : 51] Left keys [1]: [ss_customer_sk#24] Right keys [1]: [c_customer_sk#29] Join condition: None -(110) Project [codegen id : 54] +(116) Project [codegen id : 51] Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] -(111) SortAggregate +(117) HashAggregate [codegen id : 51] Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] Keys [1]: [c_customer_sk#29] +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#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] + +(118) HashAggregate [codegen id : 51] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#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))#31] -Results [2]: [c_customer_sk#29, 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))#31 AS ssales#32] +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))#35] +Results [2]: [c_customer_sk#29, 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 ssales#36] -(112) Filter [codegen id : 55] -Input [2]: [c_customer_sk#29, ssales#32] -Condition : (isnotnull(ssales#32) AND (cast(ssales#32 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#33, [id=#34] as decimal(32,6)))), DecimalType(38,8), true))) +(119) Filter [codegen id : 51] +Input [2]: [c_customer_sk#29, ssales#36] +Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) -(113) Project [codegen id : 55] +(120) Project [codegen id : 51] Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, ssales#32] +Input [2]: [c_customer_sk#29, ssales#36] + +(121) Sort [codegen id : 51] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin [codegen id : 56] -Left keys [1]: [c_customer_sk#56] +(122) SortMergeJoin [codegen id : 52] +Left keys [1]: [c_customer_sk#60] Right keys [1]: [c_customer_sk#29] Join condition: None -(115) SortMergeJoin [codegen id : 57] -Left keys [1]: [ws_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#56] +(123) SortMergeJoin [codegen id : 53] +Left keys [1]: [ws_bill_customer_sk#52] +Right keys [1]: [c_customer_sk#60] Join condition: None -(116) Project [codegen id : 57] -Output [4]: [ws_quantity#49, ws_list_price#50, c_first_name#57, c_last_name#58] -Input [6]: [ws_bill_customer_sk#48, ws_quantity#49, ws_list_price#50, c_customer_sk#56, c_first_name#57, c_last_name#58] +(124) Project [codegen id : 53] +Output [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] +Input [6]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, c_customer_sk#60, c_first_name#61, c_last_name#62] -(117) HashAggregate [codegen id : 57] -Input [4]: [ws_quantity#49, ws_list_price#50, c_first_name#57, c_last_name#58] -Keys [2]: [c_last_name#58, c_first_name#57] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#59, isEmpty#60] -Results [4]: [c_last_name#58, c_first_name#57, sum#61, isEmpty#62] +(125) HashAggregate [codegen id : 53] +Input [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] +Keys [2]: [c_last_name#62, c_first_name#61] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] -(118) Exchange -Input [4]: [c_last_name#58, c_first_name#57, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_last_name#58, c_first_name#57, 5), ENSURE_REQUIREMENTS, [id=#63] +(126) Exchange +Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +Arguments: hashpartitioning(c_last_name#62, c_first_name#61, 5), ENSURE_REQUIREMENTS, [id=#67] -(119) HashAggregate [codegen id : 58] -Input [4]: [c_last_name#58, c_first_name#57, sum#61, isEmpty#62] -Keys [2]: [c_last_name#58, c_first_name#57] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))#64] -Results [3]: [c_last_name#58, c_first_name#57, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#49 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#50 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#65] +(127) HashAggregate [codegen id : 54] +Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +Keys [2]: [c_last_name#62, c_first_name#61] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68] +Results [3]: [c_last_name#62, c_first_name#61, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68 AS sales#69] -(120) Union +(128) Union -(121) TakeOrderedAndProject -Input [3]: [c_last_name#38, c_first_name#37, sales#46] -Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, sales#46 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, sales#46] +(129) TakeOrderedAndProject +Input [3]: [c_last_name#42, c_first_name#41, sales#50] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (126) -+- * Project (125) - +- * Filter (124) - +- * ColumnarToRow (123) - +- Scan parquet default.date_dim (122) +BroadcastExchange (134) ++- * Project (133) + +- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet default.date_dim (130) -(122) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#66, d_moy#67] +(130) Scan parquet default.date_dim +Output [3]: [d_date_sk#39, d_year#70, d_moy#71] 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 -(123) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#66, d_moy#67] +(131) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] -(124) Filter [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#66, d_moy#67] -Condition : ((((isnotnull(d_year#66) AND isnotnull(d_moy#67)) AND (d_year#66 = 2000)) AND (d_moy#67 = 2)) AND isnotnull(d_date_sk#35)) +(132) Filter [codegen id : 1] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] +Condition : ((((isnotnull(d_year#70) AND isnotnull(d_moy#71)) AND (d_year#70 = 2000)) AND (d_moy#71 = 2)) AND isnotnull(d_date_sk#39)) -(125) Project [codegen id : 1] -Output [1]: [d_date_sk#35] -Input [3]: [d_date_sk#35, d_year#66, d_moy#67] +(133) Project [codegen id : 1] +Output [1]: [d_date_sk#39] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] -(126) BroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] +(134) BroadcastExchange +Input [1]: [d_date_sk#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (131) -+- * Project (130) - +- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +BroadcastExchange (139) ++- * Project (138) + +- * Filter (137) + +- * ColumnarToRow (136) + +- Scan parquet default.date_dim (135) -(127) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#69] +(135) Scan parquet default.date_dim +Output [3]: [d_date_sk#11, d_date#12, d_year#73] 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 -(128) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#69] +(136) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] -(129) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#69] -Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +(137) Filter [codegen id : 1] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] +Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) -(130) Project [codegen id : 1] +(138) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#69] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] -(131) BroadcastExchange +(139) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] - -Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#33, [id=#34] -* HashAggregate (147) -+- Exchange (146) - +- * HashAggregate (145) - +- SortAggregate (144) - +- * Project (143) - +- * SortMergeJoin Inner (142) - :- * Sort (139) - : +- Exchange (138) - : +- * Project (137) - : +- * BroadcastHashJoin Inner BuildRight (136) - : :- * Filter (134) - : : +- * ColumnarToRow (133) - : : +- Scan parquet default.store_sales (132) - : +- ReusedExchange (135) - +- * Sort (141) - +- ReusedExchange (140) - - -(132) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#74] + +Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#37, [id=#38] +* HashAggregate (156) ++- Exchange (155) + +- * HashAggregate (154) + +- * HashAggregate (153) + +- * HashAggregate (152) + +- * Project (151) + +- * SortMergeJoin Inner (150) + :- * Sort (147) + : +- Exchange (146) + : +- * Project (145) + : +- * BroadcastHashJoin Inner BuildRight (144) + : :- * Filter (142) + : : +- * ColumnarToRow (141) + : : +- Scan parquet default.store_sales (140) + : +- ReusedExchange (143) + +- * Sort (149) + +- ReusedExchange (148) + + +(140) Scan parquet default.store_sales +Output [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#74), dynamicpruningexpression(ss_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74] +(141) ColumnarToRow [codegen id : 2] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] -(134) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74] -Condition : isnotnull(ss_customer_sk#71) +(142) Filter [codegen id : 2] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_customer_sk#75) -(135) ReusedExchange [Reuses operator id: 152] -Output [1]: [d_date_sk#76] +(143) ReusedExchange [Reuses operator id: 161] +Output [1]: [d_date_sk#80] -(136) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#74] -Right keys [1]: [d_date_sk#76] +(144) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#80] Join condition: None -(137) Project [codegen id : 2] -Output [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] -Input [5]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, ss_sold_date_sk#74, d_date_sk#76] +(145) Project [codegen id : 2] +Output [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Input [5]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78, d_date_sk#80] -(138) Exchange -Input [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] -Arguments: hashpartitioning(ss_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#77] +(146) Exchange +Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Arguments: hashpartitioning(ss_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] -(139) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] -Arguments: [ss_customer_sk#71 ASC NULLS FIRST], false, 0 +(147) Sort [codegen id : 3] +Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Arguments: [ss_customer_sk#75 ASC NULLS FIRST], false, 0 -(140) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#78] +(148) ReusedExchange [Reuses operator id: 39] +Output [1]: [c_customer_sk#82] -(141) Sort [codegen id : 5] -Input [1]: [c_customer_sk#78] -Arguments: [c_customer_sk#78 ASC NULLS FIRST], false, 0 +(149) Sort [codegen id : 5] +Input [1]: [c_customer_sk#82] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 -(142) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#71] -Right keys [1]: [c_customer_sk#78] +(150) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None -(143) Project [codegen id : 6] -Output [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#78] -Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, c_customer_sk#78] - -(144) SortAggregate -Input [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#78] -Keys [1]: [c_customer_sk#78] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79 AS csales#80] - -(145) HashAggregate [codegen id : 7] -Input [1]: [csales#80] +(151) Project [codegen id : 6] +Output [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, c_customer_sk#82] + +(152) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] +Keys [1]: [c_customer_sk#82] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [3]: [c_customer_sk#82, sum#85, isEmpty#86] + +(153) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#82, sum#85, isEmpty#86] +Keys [1]: [c_customer_sk#82] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] + +(154) HashAggregate [codegen id : 6] +Input [1]: [csales#88] Keys: [] -Functions [1]: [partial_max(csales#80)] -Aggregate Attributes [1]: [max#81] -Results [1]: [max#82] +Functions [1]: [partial_max(csales#88)] +Aggregate Attributes [1]: [max#89] +Results [1]: [max#90] -(146) Exchange -Input [1]: [max#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] +(155) Exchange +Input [1]: [max#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] -(147) HashAggregate [codegen id : 8] -Input [1]: [max#82] +(156) HashAggregate [codegen id : 7] +Input [1]: [max#90] Keys: [] -Functions [1]: [max(csales#80)] -Aggregate Attributes [1]: [max(csales#80)#84] -Results [1]: [max(csales#80)#84 AS tpcds_cmax#85] +Functions [1]: [max(csales#88)] +Aggregate Attributes [1]: [max(csales#88)#92] +Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] -Subquery:4 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (152) -+- * Project (151) - +- * Filter (150) - +- * ColumnarToRow (149) - +- Scan parquet default.date_dim (148) +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 +BroadcastExchange (161) ++- * Project (160) + +- * Filter (159) + +- * ColumnarToRow (158) + +- Scan parquet default.date_dim (157) -(148) Scan parquet default.date_dim -Output [2]: [d_date_sk#76, d_year#86] +(157) Scan parquet default.date_dim +Output [2]: [d_date_sk#80, d_year#94] 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 -(149) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#76, d_year#86] +(158) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#80, d_year#94] -(150) Filter [codegen id : 1] -Input [2]: [d_date_sk#76, d_year#86] -Condition : (d_year#86 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#76)) +(159) Filter [codegen id : 1] +Input [2]: [d_date_sk#80, d_year#94] +Condition : (d_year#94 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#80)) -(151) Project [codegen id : 1] -Output [1]: [d_date_sk#76] -Input [2]: [d_date_sk#76, d_year#86] +(160) Project [codegen id : 1] +Output [1]: [d_date_sk#80] +Input [2]: [d_date_sk#80, d_year#94] -(152) BroadcastExchange -Input [1]: [d_date_sk#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#87] +(161) BroadcastExchange +Input [1]: [d_date_sk#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#95] -Subquery:5 Hosting operator id = 62 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] +Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] -Subquery:6 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#55 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] +Subquery:7 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] -Subquery:8 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#34] +Subquery:8 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] 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 7c5b18e56fa5f..1cdf12e0cc261 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (29) + WholeStageCodegen (27) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (28) + WholeStageCodegen (26) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (17) Project [cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] @@ -81,19 +81,19 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (16) - Project [c_customer_sk] - Filter [ssales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [max] [max(csales),tpcds_cmax,max] - InputAdapter - Exchange #10 - WholeStageCodegen (7) - HashAggregate [csales] [max,max] - InputAdapter - SortAggregate [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] - WholeStageCodegen (6) + WholeStageCodegen (15) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (7) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #10 + 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] InputAdapter @@ -123,9 +123,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 - InputAdapter - SortAggregate [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)),ssales,sum,isEmpty] - WholeStageCodegen (15) + 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)),ssales,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 @@ -152,72 +151,72 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - WholeStageCodegen (27) + WholeStageCodegen (25) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (19) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #13 - WholeStageCodegen (19) + WholeStageCodegen (18) Filter [c_customer_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - WholeStageCodegen (26) - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - InputAdapter - SortAggregate [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)),ssales,sum,isEmpty] - WholeStageCodegen (25) + WholeStageCodegen (24) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,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 (22) + WholeStageCodegen (21) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 - WholeStageCodegen (58) + WholeStageCodegen (54) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #14 - WholeStageCodegen (57) + WholeStageCodegen (53) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (47) + WholeStageCodegen (44) Project [ws_bill_customer_sk,ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (39) + WholeStageCodegen (37) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (38) + WholeStageCodegen (36) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_item_sk,item_sk] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (29) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #16 - WholeStageCodegen (30) + WholeStageCodegen (28) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (37) + WholeStageCodegen (35) Sort [item_sk] Project [item_sk] Filter [cnt] @@ -226,62 +225,62 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (34) + WholeStageCodegen (32) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk,d_date] #5 InputAdapter - WholeStageCodegen (36) + WholeStageCodegen (34) Sort [i_item_sk] InputAdapter ReusedExchange [i_item_sk,i_item_desc] #7 InputAdapter - WholeStageCodegen (45) - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - InputAdapter - SortAggregate [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)),ssales,sum,isEmpty] - WholeStageCodegen (44) + WholeStageCodegen (42) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,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 (41) + WholeStageCodegen (39) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - WholeStageCodegen (56) + WholeStageCodegen (52) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (49) + WholeStageCodegen (46) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 InputAdapter - WholeStageCodegen (55) - Project [c_customer_sk] - Filter [ssales] - ReusedSubquery [tpcds_cmax] #3 - InputAdapter - SortAggregate [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)),ssales,sum,isEmpty] - WholeStageCodegen (54) + WholeStageCodegen (51) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),ssales,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 (51) + WholeStageCodegen (48) Sort [ss_customer_sk] InputAdapter ReusedExchange [ss_customer_sk,ss_quantity,ss_sales_price] #8 InputAdapter - WholeStageCodegen (53) + WholeStageCodegen (50) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk] #9 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 b8826eb3ec175..b15ae61d824d4 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,62 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (58) -+- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- SortAggregate (54) - +- * Project (53) - +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildLeft (41) - : :- BroadcastExchange (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.customer_address (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.store (4) - : +- * HashAggregate (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * SortMergeJoin Inner (37) - : :- * Sort (31) - : : +- Exchange (30) - : : +- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- Union (19) - : : : : :- * Project (14) - : : : : : +- * Filter (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- Scan parquet default.catalog_sales (11) - : : : : +- * Project (18) - : : : : +- * Filter (17) - : : : : +- * ColumnarToRow (16) - : : : : +- Scan parquet default.web_sales (15) - : : : +- ReusedExchange (20) - : : +- BroadcastExchange (27) - : : +- * Project (26) - : : +- * Filter (25) - : : +- * ColumnarToRow (24) - : : +- Scan parquet default.item (23) - : +- * Sort (36) - : +- Exchange (35) - : +- * Filter (34) - : +- * ColumnarToRow (33) - : +- Scan parquet default.customer (32) - +- * Sort (51) - +- Exchange (50) - +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.store_sales (44) - +- ReusedExchange (47) +TakeOrderedAndProject (59) ++- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- * HashAggregate (54) + +- * Project (53) + +- * SortMergeJoin Inner (52) + :- * Sort (43) + : +- * Project (42) + : +- * BroadcastHashJoin Inner BuildLeft (41) + : :- BroadcastExchange (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.customer_address (1) + : : +- BroadcastExchange (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet default.store (4) + : +- * HashAggregate (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * SortMergeJoin Inner (37) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- Union (19) + : : : : :- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.catalog_sales (11) + : : : : +- * Project (18) + : : : : +- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- ReusedExchange (20) + : : +- BroadcastExchange (27) + : : +- * Project (26) + : : +- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.item (23) + : +- * Sort (36) + : +- Exchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer (32) + +- * Sort (51) + +- Exchange (50) + +- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Filter (46) + : +- * ColumnarToRow (45) + : +- Scan parquet default.store_sales (44) + +- ReusedExchange (47) (1) Scan parquet default.customer_address @@ -144,7 +145,7 @@ Input [3]: [ws_item_sk#15, ws_bill_customer_sk#16, ws_sold_date_sk#17] (19) Union -(20) ReusedExchange [Reuses operator id: 63] +(20) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#21] (21) BroadcastHashJoin [codegen id : 7] @@ -268,7 +269,7 @@ Input [3]: [ss_customer_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] Input [3]: [ss_customer_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] Condition : isnotnull(ss_customer_sk#30) -(47) ReusedExchange [Reuses operator id: 68] +(47) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#34] (48) BroadcastHashJoin [codegen id : 13] @@ -297,189 +298,196 @@ Join condition: None Output [2]: [c_customer_sk#27, ss_ext_sales_price#31] Input [3]: [c_customer_sk#27, ss_customer_sk#30, ss_ext_sales_price#31] -(54) SortAggregate +(54) HashAggregate [codegen id : 15] Input [2]: [c_customer_sk#27, ss_ext_sales_price#31] Keys [1]: [c_customer_sk#27] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#31))] +Aggregate Attributes [1]: [sum#36] +Results [2]: [c_customer_sk#27, sum#37] + +(55) HashAggregate [codegen id : 15] +Input [2]: [c_customer_sk#27, sum#37] +Keys [1]: [c_customer_sk#27] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#31))#36] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#36,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#37] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#31))#38] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#38,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#39] -(55) HashAggregate [codegen id : 16] -Input [1]: [segment#37] -Keys [1]: [segment#37] +(56) HashAggregate [codegen id : 15] +Input [1]: [segment#39] +Keys [1]: [segment#39] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#38] -Results [2]: [segment#37, count#39] +Aggregate Attributes [1]: [count#40] +Results [2]: [segment#39, count#41] -(56) Exchange -Input [2]: [segment#37, count#39] -Arguments: hashpartitioning(segment#37, 5), ENSURE_REQUIREMENTS, [id=#40] +(57) Exchange +Input [2]: [segment#39, count#41] +Arguments: hashpartitioning(segment#39, 5), ENSURE_REQUIREMENTS, [id=#42] -(57) HashAggregate [codegen id : 17] -Input [2]: [segment#37, count#39] -Keys [1]: [segment#37] +(58) HashAggregate [codegen id : 16] +Input [2]: [segment#39, count#41] +Keys [1]: [segment#39] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#41] -Results [3]: [segment#37, count(1)#41 AS num_customers#42, (segment#37 * 50) AS segment_base#43] +Aggregate Attributes [1]: [count(1)#43] +Results [3]: [segment#39, count(1)#43 AS num_customers#44, (segment#39 * 50) AS segment_base#45] -(58) TakeOrderedAndProject -Input [3]: [segment#37, num_customers#42, segment_base#43] -Arguments: 100, [segment#37 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#37, num_customers#42, segment_base#43] +(59) TakeOrderedAndProject +Input [3]: [segment#39, num_customers#44, segment_base#45] +Arguments: 100, [segment#39 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#39, num_customers#44, segment_base#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (63) -+- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet default.date_dim (59) +BroadcastExchange (64) ++- * Project (63) + +- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet default.date_dim (60) -(59) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#44, d_moy#45] +(60) Scan parquet default.date_dim +Output [3]: [d_date_sk#21, d_year#46, d_moy#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#21, d_year#44, d_moy#45] +(61) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#21, d_year#46, d_moy#47] -(61) Filter [codegen id : 1] -Input [3]: [d_date_sk#21, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 = 12)) AND (d_year#44 = 1998)) AND isnotnull(d_date_sk#21)) +(62) Filter [codegen id : 1] +Input [3]: [d_date_sk#21, d_year#46, d_moy#47] +Condition : ((((isnotnull(d_moy#47) AND isnotnull(d_year#46)) AND (d_moy#47 = 12)) AND (d_year#46 = 1998)) AND isnotnull(d_date_sk#21)) -(62) Project [codegen id : 1] +(63) Project [codegen id : 1] Output [1]: [d_date_sk#21] -Input [3]: [d_date_sk#21, d_year#44, d_moy#45] +Input [3]: [d_date_sk#21, d_year#46, d_moy#47] -(63) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#17 IN dynamicpruning#11 Subquery:3 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 -BroadcastExchange (68) -+- * Project (67) - +- * Filter (66) - +- * ColumnarToRow (65) - +- Scan parquet default.date_dim (64) +BroadcastExchange (69) ++- * Project (68) + +- * Filter (67) + +- * ColumnarToRow (66) + +- Scan parquet default.date_dim (65) -(64) Scan parquet default.date_dim -Output [2]: [d_date_sk#34, d_month_seq#47] +(65) Scan parquet default.date_dim +Output [2]: [d_date_sk#34, d_month_seq#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_month_seq#47] +(66) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#34, d_month_seq#49] -(66) Filter [codegen id : 1] -Input [2]: [d_date_sk#34, d_month_seq#47] -Condition : (((isnotnull(d_month_seq#47) AND (d_month_seq#47 >= Subquery scalar-subquery#48, [id=#49])) AND (d_month_seq#47 <= Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#34)) +(67) Filter [codegen id : 1] +Input [2]: [d_date_sk#34, d_month_seq#49] +Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= Subquery scalar-subquery#50, [id=#51])) AND (d_month_seq#49 <= Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#34)) -(67) Project [codegen id : 1] +(68) Project [codegen id : 1] Output [1]: [d_date_sk#34] -Input [2]: [d_date_sk#34, d_month_seq#47] +Input [2]: [d_date_sk#34, d_month_seq#49] -(68) BroadcastExchange +(69) BroadcastExchange Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] -Subquery:4 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (75) -+- Exchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * Filter (71) - +- * ColumnarToRow (70) - +- Scan parquet default.date_dim (69) +Subquery:4 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#50, [id=#51] +* HashAggregate (76) ++- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * Filter (72) + +- * ColumnarToRow (71) + +- Scan parquet default.date_dim (70) -(69) Scan parquet default.date_dim -Output [3]: [d_month_seq#53, d_year#54, d_moy#55] +(70) Scan parquet default.date_dim +Output [3]: [d_month_seq#55, d_year#56, d_moy#57] 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 -(70) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#53, d_year#54, d_moy#55] +(71) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] -(71) Filter [codegen id : 1] -Input [3]: [d_month_seq#53, d_year#54, d_moy#55] -Condition : (((isnotnull(d_year#54) AND isnotnull(d_moy#55)) AND (d_year#54 = 1998)) AND (d_moy#55 = 12)) +(72) Filter [codegen id : 1] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) -(72) Project [codegen id : 1] -Output [1]: [(d_month_seq#53 + 1) AS (d_month_seq + 1)#56] -Input [3]: [d_month_seq#53, d_year#54, d_moy#55] +(73) Project [codegen id : 1] +Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] -(73) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#56] -Keys [1]: [(d_month_seq + 1)#56] +(74) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#56] +Results [1]: [(d_month_seq + 1)#58] -(74) Exchange -Input [1]: [(d_month_seq + 1)#56] -Arguments: hashpartitioning((d_month_seq + 1)#56, 5), ENSURE_REQUIREMENTS, [id=#57] +(75) Exchange +Input [1]: [(d_month_seq + 1)#58] +Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] -(75) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#56] -Keys [1]: [(d_month_seq + 1)#56] +(76) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#56] +Results [1]: [(d_month_seq + 1)#58] -Subquery:5 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* HashAggregate (82) -+- Exchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet default.date_dim (76) +Subquery:5 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* HashAggregate (83) ++- Exchange (82) + +- * HashAggregate (81) + +- * Project (80) + +- * Filter (79) + +- * ColumnarToRow (78) + +- Scan parquet default.date_dim (77) -(76) Scan parquet default.date_dim -Output [3]: [d_month_seq#58, d_year#59, d_moy#60] +(77) Scan parquet default.date_dim +Output [3]: [d_month_seq#60, d_year#61, d_moy#62] 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 -(77) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#58, d_year#59, d_moy#60] +(78) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] -(78) Filter [codegen id : 1] -Input [3]: [d_month_seq#58, d_year#59, d_moy#60] -Condition : (((isnotnull(d_year#59) AND isnotnull(d_moy#60)) AND (d_year#59 = 1998)) AND (d_moy#60 = 12)) +(79) Filter [codegen id : 1] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) -(79) Project [codegen id : 1] -Output [1]: [(d_month_seq#58 + 3) AS (d_month_seq + 3)#61] -Input [3]: [d_month_seq#58, d_year#59, d_moy#60] +(80) Project [codegen id : 1] +Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] -(80) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#61] -Keys [1]: [(d_month_seq + 3)#61] +(81) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#61] +Results [1]: [(d_month_seq + 3)#63] -(81) Exchange -Input [1]: [(d_month_seq + 3)#61] -Arguments: hashpartitioning((d_month_seq + 3)#61, 5), ENSURE_REQUIREMENTS, [id=#62] +(82) Exchange +Input [1]: [(d_month_seq + 3)#63] +Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] -(82) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#61] -Keys [1]: [(d_month_seq + 3)#61] +(83) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#61] +Results [1]: [(d_month_seq + 3)#63] 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 ce221379977e4..77fb803fcf3a5 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 @@ -1,136 +1,135 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (17) + WholeStageCodegen (16) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (16) + WholeStageCodegen (15) HashAggregate [segment] [count,count] - InputAdapter - SortAggregate [c_customer_sk,ss_ext_sales_price] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - WholeStageCodegen (15) - Project [c_customer_sk,ss_ext_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] - InputAdapter - WholeStageCodegen (11) - Sort [c_customer_sk] - Project [c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ca_address_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Filter [ca_address_sk,ca_county,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [s_county,s_state] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_county,s_state] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + Project [c_customer_sk,ss_ext_sales_price] + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_customer_sk] + Project [c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ca_address_sk] + BroadcastHashJoin [ca_county,ca_state,s_county,s_state] + Filter [ca_address_sk,ca_county,ca_state] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [s_county,s_state] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_county,s_state] + HashAggregate [c_customer_sk,c_current_addr_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] + Project [c_customer_sk,c_current_addr_sk] + SortMergeJoin [customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (8) + Sort [customer_sk] + InputAdapter + Exchange [customer_sk] #4 + WholeStageCodegen (7) + Project [customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + Project [customer_sk,item_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (3) + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + 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] + WholeStageCodegen (4) + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + 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 + WholeStageCodegen (10) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #7 + WholeStageCodegen (9) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (14) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #8 + WholeStageCodegen (13) + Project [ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (8) - Sort [customer_sk] - InputAdapter - Exchange [customer_sk] #4 - WholeStageCodegen (7) - Project [customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - Project [customer_sk,item_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] + Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #3 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 1)] InputAdapter - Union - WholeStageCodegen (3) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - 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] - WholeStageCodegen (4) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Exchange [(d_month_seq + 1)] #10 + WholeStageCodegen (1) + HashAggregate [(d_month_seq + 1)] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 3)] InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - 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 - WholeStageCodegen (10) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #7 - WholeStageCodegen (9) - Filter [c_customer_sk,c_current_addr_sk] + Exchange [(d_month_seq + 3)] #11 + WholeStageCodegen (1) + HashAggregate [(d_month_seq + 3)] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (14) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #8 - WholeStageCodegen (13) - Project [ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] - InputAdapter - Exchange [(d_month_seq + 1)] #10 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 1)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] - InputAdapter - Exchange [(d_month_seq + 3)] #11 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 3)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #9 + Scan parquet default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index ddad6bf77e606..da7b19cb7d88b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- SortAggregate (41) + +- * HashAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] -(41) SortAggregate +(41) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 12] +(42) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 842c49b31045d..10f43db99224b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -1,76 +1,74 @@ -WholeStageCodegen (13) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - InputAdapter - SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - WholeStageCodegen (11) - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + Project [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] + BroadcastExchange #5 + WholeStageCodegen (8) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [d_date_sk] - Filter [d_date,d_date_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.web_site [web_site_sk,web_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 9a80843dd06fd..1c1f76169ca6a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -3,7 +3,7 @@ +- Exchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- SortAggregate (41) + +- * HashAggregate (41) +- * Project (40) +- * BroadcastHashJoin Inner BuildRight (39) :- * Project (33) @@ -221,21 +221,21 @@ Join condition: None Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] -(41) SortAggregate +(41) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(42) HashAggregate [codegen id : 12] +(42) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] Results [3]: [ws_order_number#5, sum#28, sum#29] -(43) HashAggregate [codegen id : 12] +(43) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] @@ -246,7 +246,7 @@ Results [3]: [sum#28, sum#29, count#31] Input [3]: [sum#28, sum#29, count#31] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] -(45) HashAggregate [codegen id : 13] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 7f92108322863..481c9e7c3ff4f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,76 +1,74 @@ -WholeStageCodegen (13) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - InputAdapter - SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - WholeStageCodegen (11) - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (5) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (6) - Project [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + Project [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #5 + WholeStageCodegen (8) + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] + BroadcastExchange #6 + WholeStageCodegen (9) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] + Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index c3405bb20427f..1a24233541a26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -3,7 +3,7 @@ +- Exchange (56) +- * HashAggregate (55) +- * HashAggregate (54) - +- SortAggregate (53) + +- * HashAggregate (53) +- * Project (52) +- * BroadcastHashJoin Inner BuildRight (51) :- * Project (45) @@ -283,21 +283,21 @@ Join condition: None Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#24] -(53) SortAggregate +(53) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(54) HashAggregate [codegen id : 21] +(54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(55) HashAggregate [codegen id : 21] +(55) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] @@ -308,7 +308,7 @@ Results [3]: [sum#29, sum#30, count#32] Input [3]: [sum#29, sum#30, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(57) HashAggregate [codegen id : 22] +(57) HashAggregate [codegen id : 21] Input [3]: [sum#29, sum#30, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 4d6c1fc775892..c67b17c79364c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -1,103 +1,101 @@ -WholeStageCodegen (22) +WholeStageCodegen (21) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (21) + WholeStageCodegen (20) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - InputAdapter - SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - WholeStageCodegen (20) - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] InputAdapter - WholeStageCodegen (13) - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] - InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (15) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] - ColumnarToRow + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) + Project [wr_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (13) + SortMergeJoin [wr_order_number,ws_order_number] InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + WholeStageCodegen (10) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + Project [wr_order_number] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (15) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] + BroadcastExchange #5 + WholeStageCodegen (17) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] + Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - Project [d_date_sk] - Filter [d_date,d_date_sk] + BroadcastExchange #6 + WholeStageCodegen (18) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Scan parquet default.web_site [web_site_sk,web_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (19) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index bf278bd4f2ef5..79519bfde99f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -3,7 +3,7 @@ +- Exchange (57) +- * HashAggregate (56) +- * HashAggregate (55) - +- SortAggregate (54) + +- * HashAggregate (54) +- * Project (53) +- * BroadcastHashJoin Inner BuildRight (52) :- * Project (46) @@ -288,21 +288,21 @@ Join condition: None Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#24] -(54) SortAggregate +(54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(55) HashAggregate [codegen id : 21] +(55) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] Results [3]: [ws_order_number#4, sum#29, sum#30] -(56) HashAggregate [codegen id : 21] +(56) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] @@ -313,7 +313,7 @@ Results [3]: [sum#29, sum#30, count#32] Input [3]: [sum#29, sum#30, count#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] -(58) HashAggregate [codegen id : 22] +(58) HashAggregate [codegen id : 21] Input [3]: [sum#29, sum#30, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index ebe5446364fc6..18a866c459e79 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,104 +1,102 @@ -WholeStageCodegen (22) +WholeStageCodegen (21) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (21) + WholeStageCodegen (20) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - InputAdapter - SortAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - WholeStageCodegen (20) - Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] - Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - SortMergeJoin [ws_order_number,wr_order_number] - InputAdapter - WholeStageCodegen (8) - SortMergeJoin [ws_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (2) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #2 - WholeStageCodegen (1) - Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ws_order_number] - InputAdapter - Exchange [ws_order_number] #3 - WholeStageCodegen (3) - Project [ws_warehouse_sk,ws_order_number] - Filter [ws_order_number,ws_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - WholeStageCodegen (16) - Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] - InputAdapter - WholeStageCodegen (10) - Sort [wr_order_number] - InputAdapter - Exchange [wr_order_number] #4 - WholeStageCodegen (9) - Project [wr_order_number] - Filter [wr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] InputAdapter - WholeStageCodegen (15) - Project [ws_order_number] - SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - WholeStageCodegen (12) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + Project [ws_warehouse_sk,ws_order_number] + Filter [ws_order_number,ws_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] InputAdapter - WholeStageCodegen (14) - Sort [ws_order_number] - InputAdapter - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (17) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (10) + Sort [wr_order_number] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + Project [wr_order_number] + Filter [wr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (15) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (14) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (18) - Project [ca_address_sk] - Filter [ca_state,ca_address_sk] + BroadcastExchange #5 + WholeStageCodegen (17) + Project [d_date_sk] + Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - Project [web_site_sk] - Filter [web_company_name,web_site_sk] + BroadcastExchange #6 + WholeStageCodegen (18) + Project [ca_address_sk] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_sk,web_company_name] + Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (19) + Project [web_site_sk] + Filter [web_company_name,web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt index cf42329efa257..19240a79cc91c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt @@ -68,7 +68,7 @@ : : : : : : : : : : : : : : : : +- * Filter (31) : : : : : : : : : : : : : : : : +- * HashAggregate (30) : : : : : : : : : : : : : : : : +- Exchange (29) - : : : : : : : : : : : : : : : : +- SortAggregate (28) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) : : : : : : : : : : : : : : : : +- * Project (27) : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (26) : : : : : : : : : : : : : : : : :- * Sort (19) @@ -329,7 +329,7 @@ Join condition: None Output [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Input [8]: [cs_item_sk#19, cs_order_number#20, cs_ext_list_price#21, cr_item_sk#24, cr_order_number#25, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] -(28) SortAggregate +(28) HashAggregate [codegen id : 9] Input [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Keys [1]: [cs_item_sk#19] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#21)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt index 2c2e97ce2b0ff..b5ebf7af31bed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt @@ -116,8 +116,8 @@ WholeStageCodegen (88) HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #13 - SortAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - WholeStageCodegen (9) + WholeStageCodegen (9) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 361c3b6751927..ddaa34ab4e657 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -56,7 +56,7 @@ : : : : : : : : : : : : : : : : +- * Filter (30) : : : : : : : : : : : : : : : : +- * HashAggregate (29) : : : : : : : : : : : : : : : : +- Exchange (28) - : : : : : : : : : : : : : : : : +- SortAggregate (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (27) : : : : : : : : : : : : : : : : +- * Project (26) : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (25) : : : : : : : : : : : : : : : : :- * Sort (18) @@ -299,7 +299,7 @@ Join condition: None Output [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Input [8]: [cs_item_sk#19, cs_order_number#20, cs_ext_list_price#21, cr_item_sk#24, cr_order_number#25, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] -(27) SortAggregate +(27) HashAggregate [codegen id : 8] Input [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#26, cr_reversed_charge#27, cr_store_credit#28] Keys [1]: [cs_item_sk#19] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#21)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 52687c27e0f8a..6917f8f6c6e2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -80,8 +80,8 @@ WholeStageCodegen (54) HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #6 - SortAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - WholeStageCodegen (8) + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter 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 9b6dfc78752f2..32428fbde0016 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 @@ -694,8 +694,7 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils } test("SPARK-25497: LIMIT within whole stage codegen should not consume all the inputs") { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", - SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "false") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { // A special query that only has one partition, so there is no shuffle and the entire query // can be whole-stage-codegened. val df = spark.range(0, 1500, 1, 1).limit(10).groupBy('id).count().limit(1).filter('id >= 0)