You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2022/01/13 13:36:15 UTC
[spark] branch master updated: [SPARK-35703][SQL][FOLLOWUP] Only eliminate shuffles if partition keys contain all the join keys
This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 4b4ff4b [SPARK-35703][SQL][FOLLOWUP] Only eliminate shuffles if partition keys contain all the join keys
4b4ff4b is described below
commit 4b4ff4b130306c269fb470826b2b113caf67f8bf
Author: Wenchen Fan <we...@databricks.com>
AuthorDate: Thu Jan 13 21:35:17 2022 +0800
[SPARK-35703][SQL][FOLLOWUP] Only eliminate shuffles if partition keys contain all the join keys
### What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/32875 . Basically https://github.com/apache/spark/pull/32875 did two improvements:
1. allow bucket join even if the bucket hash function is different from Spark's shuffle hash function
2. allow bucket join even if the hash partition keys are subset of join keys.
The first improvement is the major target for implementing the SPIP "storage partition join". The second improvement is kind of a consequence of the framework refactor, which is not planned.
This PR is to disable the second improvement by default, which may introduce perf regression if there are data skew without shuffle. We need more designs to enable this improvement, like checking the ndv.
### Why are the changes needed?
Avoid perf regression
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
Closes #35138 from cloud-fan/join.
Authored-by: Wenchen Fan <we...@databricks.com>
Signed-off-by: Wenchen Fan <we...@databricks.com>
---
.../sql/catalyst/plans/physical/partitioning.scala | 54 +-
.../org/apache/spark/sql/internal/SQLConf.scala | 11 +
.../spark/sql/catalyst/ShuffleSpecSuite.scala | 21 +-
.../approved-plans-v1_4/q17.sf100/explain.txt | 325 ++++----
.../approved-plans-v1_4/q17.sf100/simplified.txt | 155 ++--
.../approved-plans-v1_4/q25.sf100/explain.txt | 325 ++++----
.../approved-plans-v1_4/q25.sf100/simplified.txt | 155 ++--
.../approved-plans-v1_4/q29.sf100/explain.txt | 361 ++++-----
.../approved-plans-v1_4/q29.sf100/simplified.txt | 157 ++--
.../approved-plans-v1_4/q47.sf100/explain.txt | 235 +++---
.../approved-plans-v1_4/q47.sf100/simplified.txt | 169 +++--
.../approved-plans-v1_4/q57.sf100/explain.txt | 235 +++---
.../approved-plans-v1_4/q57.sf100/simplified.txt | 169 +++--
.../approved-plans-v1_4/q72.sf100/explain.txt | 404 +++++-----
.../approved-plans-v1_4/q72.sf100/simplified.txt | 216 +++---
.../approved-plans-v2_7/q24.sf100/explain.txt | 104 ++-
.../approved-plans-v2_7/q24.sf100/simplified.txt | 69 +-
.../approved-plans-v2_7/q47.sf100/explain.txt | 235 +++---
.../approved-plans-v2_7/q47.sf100/simplified.txt | 169 +++--
.../approved-plans-v2_7/q51a.sf100/explain.txt | 508 +++++++------
.../approved-plans-v2_7/q51a.sf100/simplified.txt | 206 ++---
.../approved-plans-v2_7/q57.sf100/explain.txt | 235 +++---
.../approved-plans-v2_7/q57.sf100/simplified.txt | 169 +++--
.../approved-plans-v2_7/q64/explain.txt | 838 +++++++++++----------
.../approved-plans-v2_7/q64/simplified.txt | 524 ++++++-------
.../approved-plans-v2_7/q72.sf100/explain.txt | 404 +++++-----
.../approved-plans-v2_7/q72.sf100/simplified.txt | 216 +++---
.../apache/spark/sql/execution/PlannerSuite.scala | 6 +-
.../exchange/EnsureRequirementsSuite.scala | 71 +-
.../spark/sql/sources/BucketedReadSuite.scala | 3 +-
30 files changed, 3506 insertions(+), 3243 deletions(-)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 7d30ecd..ed360bb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.plans.physical
import scala.collection.mutable
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, IntegerType}
/**
@@ -380,7 +381,7 @@ trait ShuffleSpec {
/**
* Whether this shuffle spec can be used to create partitionings for the other children.
*/
- def canCreatePartitioning: Boolean = false
+ def canCreatePartitioning: Boolean
/**
* Creates a partitioning that can be used to re-partition the other side with the given
@@ -412,6 +413,11 @@ case class RangeShuffleSpec(
numPartitions: Int,
distribution: ClusteredDistribution) extends ShuffleSpec {
+ // `RangePartitioning` is not compatible with any other partitioning since it can't guarantee
+ // data are co-partitioned for all the children, as range boundaries are randomly sampled. We
+ // can't let `RangeShuffleSpec` to create a partitioning.
+ override def canCreatePartitioning: Boolean = false
+
override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
case SinglePartitionShuffleSpec => numPartitions == 1
case ShuffleSpecCollection(specs) => specs.exists(isCompatibleWith)
@@ -424,8 +430,19 @@ case class RangeShuffleSpec(
case class HashShuffleSpec(
partitioning: HashPartitioning,
distribution: ClusteredDistribution) extends ShuffleSpec {
- lazy val hashKeyPositions: Seq[mutable.BitSet] =
- createHashKeyPositions(distribution.clustering, partitioning.expressions)
+
+ /**
+ * A sequence where each element is a set of positions of the hash partition key to the cluster
+ * keys. For instance, if cluster keys are [a, b, b] and hash partition keys are [a, b], the
+ * result will be [(0), (1, 2)].
+ */
+ lazy val hashKeyPositions: Seq[mutable.BitSet] = {
+ val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+ distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+ distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+ }
+ partitioning.expressions.map(k => distKeyToPos(k.canonicalized))
+ }
override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
case SinglePartitionShuffleSpec =>
@@ -451,7 +468,20 @@ case class HashShuffleSpec(
false
}
- override def canCreatePartitioning: Boolean = true
+ override def canCreatePartitioning: Boolean = {
+ // To avoid potential data skew, we don't allow `HashShuffleSpec` to create partitioning if
+ // the hash partition keys are not the full join keys (the cluster keys). Then the planner
+ // will add shuffles with the default partitioning of `ClusteredDistribution`, which uses all
+ // the join keys.
+ if (SQLConf.get.getConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION)) {
+ partitioning.expressions.length == distribution.clustering.length &&
+ partitioning.expressions.zip(distribution.clustering).forall {
+ case (l, r) => l.semanticEquals(r)
+ }
+ } else {
+ true
+ }
+ }
override def createPartitioning(clustering: Seq[Expression]): Partitioning = {
val exprs = hashKeyPositions.map(v => clustering(v.head))
@@ -459,22 +489,6 @@ case class HashShuffleSpec(
}
override def numPartitions: Int = partitioning.numPartitions
-
- /**
- * Returns a sequence where each element is a set of positions of the key in `hashKeys` to its
- * positions in `requiredClusterKeys`. For instance, if `requiredClusterKeys` is [a, b, b] and
- * `hashKeys` is [a, b], the result will be [(0), (1, 2)].
- */
- private def createHashKeyPositions(
- requiredClusterKeys: Seq[Expression],
- hashKeys: Seq[Expression]): Seq[mutable.BitSet] = {
- val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
- requiredClusterKeys.zipWithIndex.foreach { case (distKey, distKeyPos) =>
- distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
- }
-
- hashKeys.map(k => distKeyToPos(k.canonicalized))
- }
}
case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec {
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 252dd5b..42979a6 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
@@ -396,6 +396,17 @@ object SQLConf {
.booleanConf
.createWithDefault(true)
+ val REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION =
+ buildConf("spark.sql.requireAllClusterKeysForCoPartition")
+ .internal()
+ .doc("When true, the planner requires all the clustering keys as the hash partition keys " +
+ "of the children, to eliminate the shuffles for the operator that needs its children to " +
+ "be co-partitioned, such as JOIN node. This is to avoid data skews which can lead to " +
+ "significant performance regression if shuffles are eliminated.")
+ .version("3.3.0")
+ .booleanConf
+ .createWithDefault(true)
+
val RADIX_SORT_ENABLED = buildConf("spark.sql.sort.enableRadixSort")
.internal()
.doc("When true, enable use of radix sort when possible. Radix sort is much faster but " +
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala
index d4d73b3..74ec949 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala
@@ -18,11 +18,12 @@
package org.apache.spark.sql.catalyst
import org.apache.spark.SparkFunSuite
-/* Implicit conversions */
import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.internal.SQLConf
-class ShuffleSpecSuite extends SparkFunSuite {
+class ShuffleSpecSuite extends SparkFunSuite with SQLHelper {
protected def checkCompatible(
left: ShuffleSpec,
right: ShuffleSpec,
@@ -349,12 +350,22 @@ class ShuffleSpecSuite extends SparkFunSuite {
test("canCreatePartitioning") {
val distribution = ClusteredDistribution(Seq($"a", $"b"))
- assert(HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution).canCreatePartitioning)
+ withSQLConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false") {
+ assert(HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution).canCreatePartitioning)
+ }
+ withSQLConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "true") {
+ assert(!HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution)
+ .canCreatePartitioning)
+ assert(HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), distribution)
+ .canCreatePartitioning)
+ }
assert(SinglePartitionShuffleSpec.canCreatePartitioning)
- assert(ShuffleSpecCollection(Seq(
+ withSQLConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false") {
+ assert(ShuffleSpecCollection(Seq(
HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution),
HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), distribution)))
- .canCreatePartitioning)
+ .canCreatePartitioning)
+ }
assert(!RangeShuffleSpec(10, distribution).canCreatePartitioning)
}
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt
index 16afa38..d61798f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt
@@ -1,50 +1,53 @@
== Physical Plan ==
-TakeOrderedAndProject (46)
-+- * HashAggregate (45)
- +- Exchange (44)
- +- * HashAggregate (43)
- +- * Project (42)
- +- * SortMergeJoin Inner (41)
- :- * Project (32)
- : +- * SortMergeJoin Inner (31)
- : :- * Sort (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.store_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.store (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (30)
- : +- Exchange (29)
- : +- * Project (28)
- : +- * BroadcastHashJoin Inner BuildRight (27)
- : :- * Filter (25)
- : : +- * ColumnarToRow (24)
- : : +- Scan parquet default.store_returns (23)
- : +- ReusedExchange (26)
- +- * Sort (40)
- +- Exchange (39)
- +- * Project (38)
- +- * BroadcastHashJoin Inner BuildRight (37)
- :- * Filter (35)
- : +- * ColumnarToRow (34)
- : +- Scan parquet default.catalog_sales (33)
- +- ReusedExchange (36)
+TakeOrderedAndProject (49)
++- * HashAggregate (48)
+ +- Exchange (47)
+ +- * HashAggregate (46)
+ +- * Project (45)
+ +- * SortMergeJoin Inner (44)
+ :- * Sort (35)
+ : +- Exchange (34)
+ : +- * Project (33)
+ : +- * SortMergeJoin Inner (32)
+ : :- * Sort (23)
+ : : +- Exchange (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.store_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.store (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (31)
+ : +- Exchange (30)
+ : +- * Project (29)
+ : +- * BroadcastHashJoin Inner BuildRight (28)
+ : :- * Filter (26)
+ : : +- * ColumnarToRow (25)
+ : : +- Scan parquet default.store_returns (24)
+ : +- ReusedExchange (27)
+ +- * Sort (43)
+ +- Exchange (42)
+ +- * Project (41)
+ +- * BroadcastHashJoin Inner BuildRight (40)
+ :- * Filter (38)
+ : +- * ColumnarToRow (37)
+ : +- Scan parquet default.catalog_sales (36)
+ +- ReusedExchange (39)
(1) Scan parquet default.store_sales
@@ -62,7 +65,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s
Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3))
-(4) ReusedExchange [Reuses operator id: 51]
+(4) ReusedExchange [Reuses operator id: 54]
Output [1]: [d_date_sk#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -140,182 +143,194 @@ Join condition: None
Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15]
Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_sk#13, i_item_id#14, i_item_desc#15]
-(22) Sort [codegen id : 7]
+(22) Exchange
+Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15]
+Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [id=#17]
+
+(23) Sort [codegen id : 8]
Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15]
Arguments: [ss_customer_sk#2 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0
-(23) Scan parquet default.store_returns
-Output [5]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21]
+(24) Scan parquet default.store_returns
+Output [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(sr_returned_date_sk#21), dynamicpruningexpression(sr_returned_date_sk#21 IN dynamicpruning#22)]
+PartitionFilters: [isnotnull(sr_returned_date_sk#22), dynamicpruningexpression(sr_returned_date_sk#22 IN dynamicpruning#23)]
PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int,sr_return_quantity:int>
-(24) ColumnarToRow [codegen id : 9]
-Input [5]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21]
+(25) ColumnarToRow [codegen id : 10]
+Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22]
-(25) Filter [codegen id : 9]
-Input [5]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21]
-Condition : ((isnotnull(sr_customer_sk#18) AND isnotnull(sr_item_sk#17)) AND isnotnull(sr_ticket_number#19))
+(26) Filter [codegen id : 10]
+Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22]
+Condition : ((isnotnull(sr_customer_sk#19) AND isnotnull(sr_item_sk#18)) AND isnotnull(sr_ticket_number#20))
-(26) ReusedExchange [Reuses operator id: 56]
-Output [1]: [d_date_sk#23]
+(27) ReusedExchange [Reuses operator id: 59]
+Output [1]: [d_date_sk#24]
-(27) BroadcastHashJoin [codegen id : 9]
-Left keys [1]: [sr_returned_date_sk#21]
-Right keys [1]: [d_date_sk#23]
+(28) BroadcastHashJoin [codegen id : 10]
+Left keys [1]: [sr_returned_date_sk#22]
+Right keys [1]: [d_date_sk#24]
Join condition: None
-(28) Project [codegen id : 9]
-Output [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20]
-Input [6]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21, d_date_sk#23]
+(29) Project [codegen id : 10]
+Output [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
+Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22, d_date_sk#24]
-(29) Exchange
-Input [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20]
-Arguments: hashpartitioning(sr_item_sk#17, 5), ENSURE_REQUIREMENTS, [id=#24]
+(30) Exchange
+Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
+Arguments: hashpartitioning(sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20, 5), ENSURE_REQUIREMENTS, [id=#25]
-(30) Sort [codegen id : 10]
-Input [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20]
-Arguments: [sr_customer_sk#18 ASC NULLS FIRST, sr_item_sk#17 ASC NULLS FIRST, sr_ticket_number#19 ASC NULLS FIRST], false, 0
+(31) Sort [codegen id : 11]
+Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
+Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST, sr_ticket_number#20 ASC NULLS FIRST], false, 0
-(31) SortMergeJoin [codegen id : 11]
+(32) SortMergeJoin [codegen id : 12]
Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4]
-Right keys [3]: [sr_customer_sk#18, sr_item_sk#17, sr_ticket_number#19]
+Right keys [3]: [sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20]
Join condition: None
-(32) Project [codegen id : 11]
-Output [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#17, sr_customer_sk#18, sr_return_quantity#20]
-Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20]
+(33) Project [codegen id : 12]
+Output [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21]
+Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
+
+(34) Exchange
+Input [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21]
+Arguments: hashpartitioning(sr_customer_sk#19, sr_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26]
-(33) Scan parquet default.catalog_sales
-Output [4]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28]
+(35) Sort [codegen id : 13]
+Input [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21]
+Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST], false, 0
+
+(36) Scan parquet default.catalog_sales
+Output [4]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(cs_sold_date_sk#28), dynamicpruningexpression(cs_sold_date_sk#28 IN dynamicpruning#22)]
+PartitionFilters: [isnotnull(cs_sold_date_sk#30), dynamicpruningexpression(cs_sold_date_sk#30 IN dynamicpruning#23)]
PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_quantity:int>
-(34) ColumnarToRow [codegen id : 13]
-Input [4]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28]
+(37) ColumnarToRow [codegen id : 15]
+Input [4]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30]
-(35) Filter [codegen id : 13]
-Input [4]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28]
-Condition : (isnotnull(cs_bill_customer_sk#25) AND isnotnull(cs_item_sk#26))
+(38) Filter [codegen id : 15]
+Input [4]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30]
+Condition : (isnotnull(cs_bill_customer_sk#27) AND isnotnull(cs_item_sk#28))
-(36) ReusedExchange [Reuses operator id: 56]
-Output [1]: [d_date_sk#29]
+(39) ReusedExchange [Reuses operator id: 59]
+Output [1]: [d_date_sk#31]
-(37) BroadcastHashJoin [codegen id : 13]
-Left keys [1]: [cs_sold_date_sk#28]
-Right keys [1]: [d_date_sk#29]
+(40) BroadcastHashJoin [codegen id : 15]
+Left keys [1]: [cs_sold_date_sk#30]
+Right keys [1]: [d_date_sk#31]
Join condition: None
-(38) Project [codegen id : 13]
-Output [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27]
-Input [5]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28, d_date_sk#29]
+(41) Project [codegen id : 15]
+Output [3]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29]
+Input [5]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30, d_date_sk#31]
-(39) Exchange
-Input [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27]
-Arguments: hashpartitioning(cs_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30]
+(42) Exchange
+Input [3]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29]
+Arguments: hashpartitioning(cs_bill_customer_sk#27, cs_item_sk#28, 5), ENSURE_REQUIREMENTS, [id=#32]
-(40) Sort [codegen id : 14]
-Input [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27]
-Arguments: [cs_bill_customer_sk#25 ASC NULLS FIRST, cs_item_sk#26 ASC NULLS FIRST], false, 0
+(43) Sort [codegen id : 16]
+Input [3]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29]
+Arguments: [cs_bill_customer_sk#27 ASC NULLS FIRST, cs_item_sk#28 ASC NULLS FIRST], false, 0
-(41) SortMergeJoin [codegen id : 15]
-Left keys [2]: [sr_customer_sk#18, sr_item_sk#17]
-Right keys [2]: [cs_bill_customer_sk#25, cs_item_sk#26]
+(44) SortMergeJoin [codegen id : 17]
+Left keys [2]: [sr_customer_sk#19, sr_item_sk#18]
+Right keys [2]: [cs_bill_customer_sk#27, cs_item_sk#28]
Join condition: None
-(42) Project [codegen id : 15]
-Output [6]: [ss_quantity#5, sr_return_quantity#20, cs_quantity#27, s_state#10, i_item_id#14, i_item_desc#15]
-Input [10]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#17, sr_customer_sk#18, sr_return_quantity#20, cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27]
+(45) Project [codegen id : 17]
+Output [6]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#29, s_state#10, i_item_id#14, i_item_desc#15]
+Input [10]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21, cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29]
-(43) HashAggregate [codegen id : 15]
-Input [6]: [ss_quantity#5, sr_return_quantity#20, cs_quantity#27, s_state#10, i_item_id#14, i_item_desc#15]
+(46) HashAggregate [codegen id : 17]
+Input [6]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#29, s_state#10, i_item_id#14, i_item_desc#15]
Keys [3]: [i_item_id#14, i_item_desc#15, s_state#10]
-Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#20), partial_avg(sr_return_quantity#20), partial_stddev_samp(cast(sr_return_quantity#20 as double)), partial_count(cs_quantity#27), partial_avg(cs_quantity#27), partial_stddev_samp(cast(cs_quantity#27 as double))]
-Aggregate Attributes [18]: [count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48]
-Results [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#49, sum#50, count#51, n#52, avg#53, m2#54, count#55, sum#56, count#57, n#58, avg#59, m2#60, count#61, sum#62, count#63, n#64, avg#65, m2#66]
+Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#21), partial_avg(sr_return_quantity#21), partial_stddev_samp(cast(sr_return_quantity#21 as double)), partial_count(cs_quantity#29), partial_avg(cs_quantity#29), partial_stddev_samp(cast(cs_quantity#29 as double))]
+Aggregate Attributes [18]: [count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44, count#45, sum#46, count#47, n#48, avg#49, m2#50]
+Results [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62, count#63, sum#64, count#65, n#66, avg#67, m2#68]
-(44) Exchange
-Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#49, sum#50, count#51, n#52, avg#53, m2#54, count#55, sum#56, count#57, n#58, avg#59, m2#60, count#61, sum#62, count#63, n#64, avg#65, m2#66]
-Arguments: hashpartitioning(i_item_id#14, i_item_desc#15, s_state#10, 5), ENSURE_REQUIREMENTS, [id=#67]
+(47) Exchange
+Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62, count#63, sum#64, count#65, n#66, avg#67, m2#68]
+Arguments: hashpartitioning(i_item_id#14, i_item_desc#15, s_state#10, 5), ENSURE_REQUIREMENTS, [id=#69]
-(45) HashAggregate [codegen id : 16]
-Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#49, sum#50, count#51, n#52, avg#53, m2#54, count#55, sum#56, count#57, n#58, avg#59, m2#60, count#61, sum#62, count#63, n#64, avg#65, m2#66]
+(48) HashAggregate [codegen id : 18]
+Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62, count#63, sum#64, count#65, n#66, avg#67, m2#68]
Keys [3]: [i_item_id#14, i_item_desc#15, s_state#10]
-Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#20), avg(sr_return_quantity#20), stddev_samp(cast(sr_return_quantity#20 as double)), count(cs_quantity#27), avg(cs_quantity#27), stddev_samp(cast(cs_quantity#27 as double))]
-Aggregate Attributes [9]: [count(ss_quantity#5)#68, avg(ss_quantity#5)#69, stddev_samp(cast(ss_quantity#5 as double))#70, count(sr_return_quantity#20)#71, avg(sr_return_quantity#20)#72, stddev_samp(cast(sr_return_quantity#20 as double))#73, count(cs_quantity#27)#74, avg(cs_quantity#27)#75, stddev_samp(cast(cs_quantity#27 as double))#76]
-Results [15]: [i_item_id#14, i_item_desc#15, s_state#10, count(ss_quantity#5)#68 AS store_sales_quantitycount#77, avg(ss_quantity#5)#69 AS store_sales_quantityave#78, stddev_samp(cast(ss_quantity#5 as double))#70 AS store_sales_quantitystdev#79, (stddev_samp(cast(ss_quantity#5 as double))#70 / avg(ss_quantity#5)#69) AS store_sales_quantitycov#80, count(sr_return_quantity#20)#71 AS as_store_returns_quantitycount#81, avg(sr_return_quantity#20)#72 AS as_store_returns_quantityave#82, stddev_ [...]
+Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#21), avg(sr_return_quantity#21), stddev_samp(cast(sr_return_quantity#21 as double)), count(cs_quantity#29), avg(cs_quantity#29), stddev_samp(cast(cs_quantity#29 as double))]
+Aggregate Attributes [9]: [count(ss_quantity#5)#70, avg(ss_quantity#5)#71, stddev_samp(cast(ss_quantity#5 as double))#72, count(sr_return_quantity#21)#73, avg(sr_return_quantity#21)#74, stddev_samp(cast(sr_return_quantity#21 as double))#75, count(cs_quantity#29)#76, avg(cs_quantity#29)#77, stddev_samp(cast(cs_quantity#29 as double))#78]
+Results [15]: [i_item_id#14, i_item_desc#15, s_state#10, count(ss_quantity#5)#70 AS store_sales_quantitycount#79, avg(ss_quantity#5)#71 AS store_sales_quantityave#80, stddev_samp(cast(ss_quantity#5 as double))#72 AS store_sales_quantitystdev#81, (stddev_samp(cast(ss_quantity#5 as double))#72 / avg(ss_quantity#5)#71) AS store_sales_quantitycov#82, count(sr_return_quantity#21)#73 AS as_store_returns_quantitycount#83, avg(sr_return_quantity#21)#74 AS as_store_returns_quantityave#84, stddev_ [...]
-(46) TakeOrderedAndProject
-Input [15]: [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#77, store_sales_quantityave#78, store_sales_quantitystdev#79, store_sales_quantitycov#80, as_store_returns_quantitycount#81, as_store_returns_quantityave#82, as_store_returns_quantitystdev#83, store_returns_quantitycov#84, catalog_sales_quantitycount#85, catalog_sales_quantityave#86, catalog_sales_quantitystdev#87, catalog_sales_quantitycov#88]
-Arguments: 100, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#77, store_sales_quantityave#78, store_sales_quantitystdev#79, store_sales_quantitycov#80, as_store_returns_quantitycount#81, as_store_returns_quantityave#82, as_store_returns_quantitystdev#83, store_returns_quantitycov#84, catalog_sales_quantitycount#85, catalog_sales_quantityave#86, catalog_sales_quantitystdev#87 [...]
+(49) TakeOrderedAndProject
+Input [15]: [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90]
+Arguments: 100, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89 [...]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
-BroadcastExchange (51)
-+- * Project (50)
- +- * Filter (49)
- +- * ColumnarToRow (48)
- +- Scan parquet default.date_dim (47)
+BroadcastExchange (54)
++- * Project (53)
+ +- * Filter (52)
+ +- * ColumnarToRow (51)
+ +- Scan parquet default.date_dim (50)
-(47) Scan parquet default.date_dim
-Output [2]: [d_date_sk#8, d_quarter_name#89]
+(50) Scan parquet default.date_dim
+Output [2]: [d_date_sk#8, d_quarter_name#91]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_quarter_name:string>
-(48) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#8, d_quarter_name#89]
+(51) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#8, d_quarter_name#91]
-(49) Filter [codegen id : 1]
-Input [2]: [d_date_sk#8, d_quarter_name#89]
-Condition : ((isnotnull(d_quarter_name#89) AND (d_quarter_name#89 = 2001Q1)) AND isnotnull(d_date_sk#8))
+(52) Filter [codegen id : 1]
+Input [2]: [d_date_sk#8, d_quarter_name#91]
+Condition : ((isnotnull(d_quarter_name#91) AND (d_quarter_name#91 = 2001Q1)) AND isnotnull(d_date_sk#8))
-(50) Project [codegen id : 1]
+(53) Project [codegen id : 1]
Output [1]: [d_date_sk#8]
-Input [2]: [d_date_sk#8, d_quarter_name#89]
+Input [2]: [d_date_sk#8, d_quarter_name#91]
-(51) BroadcastExchange
+(54) BroadcastExchange
Input [1]: [d_date_sk#8]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92]
-Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#21 IN dynamicpruning#22
-BroadcastExchange (56)
-+- * Project (55)
- +- * Filter (54)
- +- * ColumnarToRow (53)
- +- Scan parquet default.date_dim (52)
+Subquery:2 Hosting operator id = 24 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23
+BroadcastExchange (59)
++- * Project (58)
+ +- * Filter (57)
+ +- * ColumnarToRow (56)
+ +- Scan parquet default.date_dim (55)
-(52) Scan parquet default.date_dim
-Output [2]: [d_date_sk#23, d_quarter_name#91]
+(55) Scan parquet default.date_dim
+Output [2]: [d_date_sk#24, d_quarter_name#93]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_quarter_name:string>
-(53) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#23, d_quarter_name#91]
+(56) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#24, d_quarter_name#93]
-(54) Filter [codegen id : 1]
-Input [2]: [d_date_sk#23, d_quarter_name#91]
-Condition : (d_quarter_name#91 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#23))
+(57) Filter [codegen id : 1]
+Input [2]: [d_date_sk#24, d_quarter_name#93]
+Condition : (d_quarter_name#93 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24))
-(55) Project [codegen id : 1]
-Output [1]: [d_date_sk#23]
-Input [2]: [d_date_sk#23, d_quarter_name#91]
+(58) Project [codegen id : 1]
+Output [1]: [d_date_sk#24]
+Input [2]: [d_date_sk#24, d_quarter_name#93]
-(56) BroadcastExchange
-Input [1]: [d_date_sk#23]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92]
+(59) BroadcastExchange
+Input [1]: [d_date_sk#24]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94]
-Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#28 IN dynamicpruning#22
+Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#30 IN dynamicpruning#23
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt
index b00c5da..06c8f7b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt
@@ -1,90 +1,97 @@
TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov]
- WholeStageCodegen (16)
+ WholeStageCodegen (18)
HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_retur [...]
InputAdapter
Exchange [i_item_id,i_item_desc,s_state] #1
- WholeStageCodegen (15)
+ WholeStageCodegen (17)
HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2]
Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc]
SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk]
InputAdapter
- WholeStageCodegen (11)
- Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity]
- SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
- InputAdapter
- WholeStageCodegen (7)
- Sort [ss_customer_sk,ss_item_sk,ss_ticket_number]
- Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc]
- SortMergeJoin [ss_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #2
- WholeStageCodegen (3)
- Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #3
- WholeStageCodegen (1)
- Project [d_date_sk]
- Filter [d_quarter_name,d_date_sk]
+ WholeStageCodegen (13)
+ Sort [sr_customer_sk,sr_item_sk]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_item_sk] #2
+ WholeStageCodegen (12)
+ Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity]
+ SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ WholeStageCodegen (8)
+ Sort [ss_customer_sk,ss_item_sk,ss_ticket_number]
+ InputAdapter
+ Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3
+ WholeStageCodegen (7)
+ Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc]
+ SortMergeJoin [ss_item_sk,i_item_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_item_sk] #4
+ WholeStageCodegen (3)
+ Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ Project [d_date_sk]
+ Filter [d_quarter_name,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_quarter_name]
+ InputAdapter
+ ReusedExchange [d_date_sk] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [s_store_sk]
ColumnarToRow
InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_quarter_name]
+ Scan parquet default.store [s_store_sk,s_state]
+ InputAdapter
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
InputAdapter
- ReusedExchange [d_date_sk] #3
- InputAdapter
- BroadcastExchange #4
- WholeStageCodegen (2)
- Filter [s_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store [s_store_sk,s_state]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #5
- WholeStageCodegen (5)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_item_id,i_item_desc]
- InputAdapter
- WholeStageCodegen (10)
- Sort [sr_customer_sk,sr_item_sk,sr_ticket_number]
- InputAdapter
- Exchange [sr_item_sk] #6
- WholeStageCodegen (9)
- Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity]
- BroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- Filter [sr_customer_sk,sr_item_sk,sr_ticket_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
- SubqueryBroadcast [d_date_sk] #2
- BroadcastExchange #7
- WholeStageCodegen (1)
- Project [d_date_sk]
- Filter [d_quarter_name,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_quarter_name]
- InputAdapter
- ReusedExchange [d_date_sk] #7
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_item_id,i_item_desc]
+ InputAdapter
+ WholeStageCodegen (11)
+ Sort [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8
+ WholeStageCodegen (10)
+ Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity]
+ BroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ Filter [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #9
+ WholeStageCodegen (1)
+ Project [d_date_sk]
+ Filter [d_quarter_name,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_quarter_name]
+ InputAdapter
+ ReusedExchange [d_date_sk] #9
InputAdapter
- WholeStageCodegen (14)
+ WholeStageCodegen (16)
Sort [cs_bill_customer_sk,cs_item_sk]
InputAdapter
- Exchange [cs_item_sk] #8
- WholeStageCodegen (13)
+ Exchange [cs_bill_customer_sk,cs_item_sk] #10
+ WholeStageCodegen (15)
Project [cs_bill_customer_sk,cs_item_sk,cs_quantity]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
Filter [cs_bill_customer_sk,cs_item_sk]
@@ -93,4 +100,4 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
InputAdapter
- ReusedExchange [d_date_sk] #7
+ ReusedExchange [d_date_sk] #9
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt
index cbbf3da..fc55789 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt
@@ -1,50 +1,53 @@
== Physical Plan ==
-TakeOrderedAndProject (46)
-+- * HashAggregate (45)
- +- Exchange (44)
- +- * HashAggregate (43)
- +- * Project (42)
- +- * SortMergeJoin Inner (41)
- :- * Project (32)
- : +- * SortMergeJoin Inner (31)
- : :- * Sort (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.store_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.store (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (30)
- : +- Exchange (29)
- : +- * Project (28)
- : +- * BroadcastHashJoin Inner BuildRight (27)
- : :- * Filter (25)
- : : +- * ColumnarToRow (24)
- : : +- Scan parquet default.store_returns (23)
- : +- ReusedExchange (26)
- +- * Sort (40)
- +- Exchange (39)
- +- * Project (38)
- +- * BroadcastHashJoin Inner BuildRight (37)
- :- * Filter (35)
- : +- * ColumnarToRow (34)
- : +- Scan parquet default.catalog_sales (33)
- +- ReusedExchange (36)
+TakeOrderedAndProject (49)
++- * HashAggregate (48)
+ +- Exchange (47)
+ +- * HashAggregate (46)
+ +- * Project (45)
+ +- * SortMergeJoin Inner (44)
+ :- * Sort (35)
+ : +- Exchange (34)
+ : +- * Project (33)
+ : +- * SortMergeJoin Inner (32)
+ : :- * Sort (23)
+ : : +- Exchange (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.store_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.store (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (31)
+ : +- Exchange (30)
+ : +- * Project (29)
+ : +- * BroadcastHashJoin Inner BuildRight (28)
+ : :- * Filter (26)
+ : : +- * ColumnarToRow (25)
+ : : +- Scan parquet default.store_returns (24)
+ : +- ReusedExchange (27)
+ +- * Sort (43)
+ +- Exchange (42)
+ +- * Project (41)
+ +- * BroadcastHashJoin Inner BuildRight (40)
+ :- * Filter (38)
+ : +- * ColumnarToRow (37)
+ : +- Scan parquet default.catalog_sales (36)
+ +- ReusedExchange (39)
(1) Scan parquet default.store_sales
@@ -62,7 +65,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s
Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6]
Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3))
-(4) ReusedExchange [Reuses operator id: 51]
+(4) ReusedExchange [Reuses operator id: 54]
Output [1]: [d_date_sk#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -140,182 +143,194 @@ Join condition: None
Output [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_sk#14, i_item_id#15, i_item_desc#16]
-(22) Sort [codegen id : 7]
+(22) Exchange
+Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
+Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [id=#18]
+
+(23) Sort [codegen id : 8]
Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
Arguments: [ss_customer_sk#2 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0
-(23) Scan parquet default.store_returns
-Output [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22]
+(24) Scan parquet default.store_returns
+Output [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(sr_returned_date_sk#22), dynamicpruningexpression(sr_returned_date_sk#22 IN dynamicpruning#23)]
+PartitionFilters: [isnotnull(sr_returned_date_sk#23), dynamicpruningexpression(sr_returned_date_sk#23 IN dynamicpruning#24)]
PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int,sr_net_loss:decimal(7,2)>
-(24) ColumnarToRow [codegen id : 9]
-Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22]
+(25) ColumnarToRow [codegen id : 10]
+Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23]
-(25) Filter [codegen id : 9]
-Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22]
-Condition : ((isnotnull(sr_customer_sk#19) AND isnotnull(sr_item_sk#18)) AND isnotnull(sr_ticket_number#20))
+(26) Filter [codegen id : 10]
+Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23]
+Condition : ((isnotnull(sr_customer_sk#20) AND isnotnull(sr_item_sk#19)) AND isnotnull(sr_ticket_number#21))
-(26) ReusedExchange [Reuses operator id: 56]
-Output [1]: [d_date_sk#24]
+(27) ReusedExchange [Reuses operator id: 59]
+Output [1]: [d_date_sk#25]
-(27) BroadcastHashJoin [codegen id : 9]
-Left keys [1]: [sr_returned_date_sk#22]
-Right keys [1]: [d_date_sk#24]
+(28) BroadcastHashJoin [codegen id : 10]
+Left keys [1]: [sr_returned_date_sk#23]
+Right keys [1]: [d_date_sk#25]
Join condition: None
-(28) Project [codegen id : 9]
-Output [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21]
-Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22, d_date_sk#24]
+(29) Project [codegen id : 10]
+Output [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22]
+Input [6]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25]
-(29) Exchange
-Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21]
-Arguments: hashpartitioning(sr_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#25]
+(30) Exchange
+Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22]
+Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21, 5), ENSURE_REQUIREMENTS, [id=#26]
-(30) Sort [codegen id : 10]
-Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21]
-Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST, sr_ticket_number#20 ASC NULLS FIRST], false, 0
+(31) Sort [codegen id : 11]
+Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22]
+Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST, sr_ticket_number#21 ASC NULLS FIRST], false, 0
-(31) SortMergeJoin [codegen id : 11]
+(32) SortMergeJoin [codegen id : 12]
Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4]
-Right keys [3]: [sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20]
+Right keys [3]: [sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21]
Join condition: None
-(32) Project [codegen id : 11]
-Output [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_net_loss#21]
-Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21]
+(33) Project [codegen id : 12]
+Output [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22]
+Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22]
+
+(34) Exchange
+Input [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22]
+Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#27]
+
+(35) Sort [codegen id : 13]
+Input [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22]
+Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST], false, 0
-(33) Scan parquet default.catalog_sales
-Output [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29]
+(36) Scan parquet default.catalog_sales
+Output [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(cs_sold_date_sk#29), dynamicpruningexpression(cs_sold_date_sk#29 IN dynamicpruning#23)]
+PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#24)]
PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_net_profit:decimal(7,2)>
-(34) ColumnarToRow [codegen id : 13]
-Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29]
+(37) ColumnarToRow [codegen id : 15]
+Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31]
-(35) Filter [codegen id : 13]
-Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29]
-Condition : (isnotnull(cs_bill_customer_sk#26) AND isnotnull(cs_item_sk#27))
+(38) Filter [codegen id : 15]
+Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31]
+Condition : (isnotnull(cs_bill_customer_sk#28) AND isnotnull(cs_item_sk#29))
-(36) ReusedExchange [Reuses operator id: 56]
-Output [1]: [d_date_sk#30]
+(39) ReusedExchange [Reuses operator id: 59]
+Output [1]: [d_date_sk#32]
-(37) BroadcastHashJoin [codegen id : 13]
-Left keys [1]: [cs_sold_date_sk#29]
-Right keys [1]: [d_date_sk#30]
+(40) BroadcastHashJoin [codegen id : 15]
+Left keys [1]: [cs_sold_date_sk#31]
+Right keys [1]: [d_date_sk#32]
Join condition: None
-(38) Project [codegen id : 13]
-Output [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28]
-Input [5]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29, d_date_sk#30]
+(41) Project [codegen id : 15]
+Output [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30]
+Input [5]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#32]
-(39) Exchange
-Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28]
-Arguments: hashpartitioning(cs_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#31]
+(42) Exchange
+Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30]
+Arguments: hashpartitioning(cs_bill_customer_sk#28, cs_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#33]
-(40) Sort [codegen id : 14]
-Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28]
-Arguments: [cs_bill_customer_sk#26 ASC NULLS FIRST, cs_item_sk#27 ASC NULLS FIRST], false, 0
+(43) Sort [codegen id : 16]
+Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30]
+Arguments: [cs_bill_customer_sk#28 ASC NULLS FIRST, cs_item_sk#29 ASC NULLS FIRST], false, 0
-(41) SortMergeJoin [codegen id : 15]
-Left keys [2]: [sr_customer_sk#19, sr_item_sk#18]
-Right keys [2]: [cs_bill_customer_sk#26, cs_item_sk#27]
+(44) SortMergeJoin [codegen id : 17]
+Left keys [2]: [sr_customer_sk#20, sr_item_sk#19]
+Right keys [2]: [cs_bill_customer_sk#28, cs_item_sk#29]
Join condition: None
-(42) Project [codegen id : 15]
-Output [7]: [ss_net_profit#5, sr_net_loss#21, cs_net_profit#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
-Input [11]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_net_loss#21, cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28]
+(45) Project [codegen id : 17]
+Output [7]: [ss_net_profit#5, sr_net_loss#22, cs_net_profit#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
+Input [11]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22, cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30]
-(43) HashAggregate [codegen id : 15]
-Input [7]: [ss_net_profit#5, sr_net_loss#21, cs_net_profit#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
+(46) HashAggregate [codegen id : 17]
+Input [7]: [ss_net_profit#5, sr_net_loss#22, cs_net_profit#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11]
-Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#21)), partial_sum(UnscaledValue(cs_net_profit#28))]
-Aggregate Attributes [3]: [sum#32, sum#33, sum#34]
-Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#35, sum#36, sum#37]
+Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#22)), partial_sum(UnscaledValue(cs_net_profit#30))]
+Aggregate Attributes [3]: [sum#34, sum#35, sum#36]
+Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#37, sum#38, sum#39]
-(44) Exchange
-Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#35, sum#36, sum#37]
-Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#38]
+(47) Exchange
+Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#37, sum#38, sum#39]
+Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#40]
-(45) HashAggregate [codegen id : 16]
-Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#35, sum#36, sum#37]
+(48) HashAggregate [codegen id : 18]
+Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#37, sum#38, sum#39]
Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11]
-Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#21)), sum(UnscaledValue(cs_net_profit#28))]
-Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#39, sum(UnscaledValue(sr_net_loss#21))#40, sum(UnscaledValue(cs_net_profit#28))#41]
-Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#39,17,2) AS store_sales_profit#42, MakeDecimal(sum(UnscaledValue(sr_net_loss#21))#40,17,2) AS store_returns_loss#43, MakeDecimal(sum(UnscaledValue(cs_net_profit#28))#41,17,2) AS catalog_sales_profit#44]
+Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#22)), sum(UnscaledValue(cs_net_profit#30))]
+Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#41, sum(UnscaledValue(sr_net_loss#22))#42, sum(UnscaledValue(cs_net_profit#30))#43]
+Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#41,17,2) AS store_sales_profit#44, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#42,17,2) AS store_returns_loss#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#30))#43,17,2) AS catalog_sales_profit#46]
-(46) TakeOrderedAndProject
-Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#42, store_returns_loss#43, catalog_sales_profit#44]
-Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#42, store_returns_loss#43, catalog_sales_profit#44]
+(49) TakeOrderedAndProject
+Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46]
+Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
-BroadcastExchange (51)
-+- * Project (50)
- +- * Filter (49)
- +- * ColumnarToRow (48)
- +- Scan parquet default.date_dim (47)
+BroadcastExchange (54)
++- * Project (53)
+ +- * Filter (52)
+ +- * ColumnarToRow (51)
+ +- Scan parquet default.date_dim (50)
-(47) Scan parquet default.date_dim
-Output [3]: [d_date_sk#8, d_year#45, d_moy#46]
+(50) Scan parquet default.date_dim
+Output [3]: [d_date_sk#8, d_year#47, d_moy#48]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(48) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#8, d_year#45, d_moy#46]
+(51) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#8, d_year#47, d_moy#48]
-(49) Filter [codegen id : 1]
-Input [3]: [d_date_sk#8, d_year#45, d_moy#46]
-Condition : ((((isnotnull(d_moy#46) AND isnotnull(d_year#45)) AND (d_moy#46 = 4)) AND (d_year#45 = 2001)) AND isnotnull(d_date_sk#8))
+(52) Filter [codegen id : 1]
+Input [3]: [d_date_sk#8, d_year#47, d_moy#48]
+Condition : ((((isnotnull(d_moy#48) AND isnotnull(d_year#47)) AND (d_moy#48 = 4)) AND (d_year#47 = 2001)) AND isnotnull(d_date_sk#8))
-(50) Project [codegen id : 1]
+(53) Project [codegen id : 1]
Output [1]: [d_date_sk#8]
-Input [3]: [d_date_sk#8, d_year#45, d_moy#46]
+Input [3]: [d_date_sk#8, d_year#47, d_moy#48]
-(51) BroadcastExchange
+(54) BroadcastExchange
Input [1]: [d_date_sk#8]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49]
-Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23
-BroadcastExchange (56)
-+- * Project (55)
- +- * Filter (54)
- +- * ColumnarToRow (53)
- +- Scan parquet default.date_dim (52)
+Subquery:2 Hosting operator id = 24 Hosting Expression = sr_returned_date_sk#23 IN dynamicpruning#24
+BroadcastExchange (59)
++- * Project (58)
+ +- * Filter (57)
+ +- * ColumnarToRow (56)
+ +- Scan parquet default.date_dim (55)
-(52) Scan parquet default.date_dim
-Output [3]: [d_date_sk#24, d_year#48, d_moy#49]
+(55) Scan parquet default.date_dim
+Output [3]: [d_date_sk#25, d_year#50, d_moy#51]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(53) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#24, d_year#48, d_moy#49]
+(56) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#25, d_year#50, d_moy#51]
-(54) Filter [codegen id : 1]
-Input [3]: [d_date_sk#24, d_year#48, d_moy#49]
-Condition : (((((isnotnull(d_moy#49) AND isnotnull(d_year#48)) AND (d_moy#49 >= 4)) AND (d_moy#49 <= 10)) AND (d_year#48 = 2001)) AND isnotnull(d_date_sk#24))
+(57) Filter [codegen id : 1]
+Input [3]: [d_date_sk#25, d_year#50, d_moy#51]
+Condition : (((((isnotnull(d_moy#51) AND isnotnull(d_year#50)) AND (d_moy#51 >= 4)) AND (d_moy#51 <= 10)) AND (d_year#50 = 2001)) AND isnotnull(d_date_sk#25))
-(55) Project [codegen id : 1]
-Output [1]: [d_date_sk#24]
-Input [3]: [d_date_sk#24, d_year#48, d_moy#49]
+(58) Project [codegen id : 1]
+Output [1]: [d_date_sk#25]
+Input [3]: [d_date_sk#25, d_year#50, d_moy#51]
-(56) BroadcastExchange
-Input [1]: [d_date_sk#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50]
+(59) BroadcastExchange
+Input [1]: [d_date_sk#25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52]
-Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#23
+Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#24
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt
index 0b106ce..23d7e84 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt
@@ -1,90 +1,97 @@
TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit]
- WholeStageCodegen (16)
+ WholeStageCodegen (18)
HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum]
InputAdapter
Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1
- WholeStageCodegen (15)
+ WholeStageCodegen (17)
HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum]
Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc]
SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk]
InputAdapter
- WholeStageCodegen (11)
- Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss]
- SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
- InputAdapter
- WholeStageCodegen (7)
- Sort [ss_customer_sk,ss_item_sk,ss_ticket_number]
- Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc]
- SortMergeJoin [ss_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #2
- WholeStageCodegen (3)
- Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #3
- WholeStageCodegen (1)
- Project [d_date_sk]
- Filter [d_moy,d_year,d_date_sk]
+ WholeStageCodegen (13)
+ Sort [sr_customer_sk,sr_item_sk]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_item_sk] #2
+ WholeStageCodegen (12)
+ Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss]
+ SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ WholeStageCodegen (8)
+ Sort [ss_customer_sk,ss_item_sk,ss_ticket_number]
+ InputAdapter
+ Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3
+ WholeStageCodegen (7)
+ Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc]
+ SortMergeJoin [ss_item_sk,i_item_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_item_sk] #4
+ WholeStageCodegen (3)
+ Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_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]
+ InputAdapter
+ ReusedExchange [d_date_sk] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [s_store_sk]
ColumnarToRow
InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
+ Scan parquet default.store [s_store_sk,s_store_id,s_store_name]
+ InputAdapter
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
InputAdapter
- ReusedExchange [d_date_sk] #3
- InputAdapter
- BroadcastExchange #4
- WholeStageCodegen (2)
- Filter [s_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store [s_store_sk,s_store_id,s_store_name]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #5
- WholeStageCodegen (5)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_item_id,i_item_desc]
- InputAdapter
- WholeStageCodegen (10)
- Sort [sr_customer_sk,sr_item_sk,sr_ticket_number]
- InputAdapter
- Exchange [sr_item_sk] #6
- WholeStageCodegen (9)
- Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss]
- BroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- Filter [sr_customer_sk,sr_item_sk,sr_ticket_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
- SubqueryBroadcast [d_date_sk] #2
- BroadcastExchange #7
- 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]
- InputAdapter
- ReusedExchange [d_date_sk] #7
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_item_id,i_item_desc]
+ InputAdapter
+ WholeStageCodegen (11)
+ Sort [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8
+ WholeStageCodegen (10)
+ Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss]
+ BroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ Filter [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #9
+ 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]
+ InputAdapter
+ ReusedExchange [d_date_sk] #9
InputAdapter
- WholeStageCodegen (14)
+ WholeStageCodegen (16)
Sort [cs_bill_customer_sk,cs_item_sk]
InputAdapter
- Exchange [cs_item_sk] #8
- WholeStageCodegen (13)
+ Exchange [cs_bill_customer_sk,cs_item_sk] #10
+ WholeStageCodegen (15)
Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
Filter [cs_bill_customer_sk,cs_item_sk]
@@ -93,4 +100,4 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
InputAdapter
- ReusedExchange [d_date_sk] #7
+ ReusedExchange [d_date_sk] #9
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt
index e9857b7..2214390 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt
@@ -1,50 +1,53 @@
== Physical Plan ==
-TakeOrderedAndProject (46)
-+- * HashAggregate (45)
- +- Exchange (44)
- +- * HashAggregate (43)
- +- * Project (42)
- +- * SortMergeJoin Inner (41)
- :- * Project (32)
- : +- * SortMergeJoin Inner (31)
- : :- * Sort (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.store_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.store (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (30)
- : +- Exchange (29)
- : +- * Project (28)
- : +- * BroadcastHashJoin Inner BuildRight (27)
- : :- * Filter (25)
- : : +- * ColumnarToRow (24)
- : : +- Scan parquet default.store_returns (23)
- : +- ReusedExchange (26)
- +- * Sort (40)
- +- Exchange (39)
- +- * Project (38)
- +- * BroadcastHashJoin Inner BuildRight (37)
- :- * Filter (35)
- : +- * ColumnarToRow (34)
- : +- Scan parquet default.catalog_sales (33)
- +- ReusedExchange (36)
+TakeOrderedAndProject (49)
++- * HashAggregate (48)
+ +- Exchange (47)
+ +- * HashAggregate (46)
+ +- * Project (45)
+ +- * SortMergeJoin Inner (44)
+ :- * Sort (35)
+ : +- Exchange (34)
+ : +- * Project (33)
+ : +- * SortMergeJoin Inner (32)
+ : :- * Sort (23)
+ : : +- Exchange (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.store_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.store (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (31)
+ : +- Exchange (30)
+ : +- * Project (29)
+ : +- * BroadcastHashJoin Inner BuildRight (28)
+ : :- * Filter (26)
+ : : +- * ColumnarToRow (25)
+ : : +- Scan parquet default.store_returns (24)
+ : +- ReusedExchange (27)
+ +- * Sort (43)
+ +- Exchange (42)
+ +- * Project (41)
+ +- * BroadcastHashJoin Inner BuildRight (40)
+ :- * Filter (38)
+ : +- * ColumnarToRow (37)
+ : +- Scan parquet default.catalog_sales (36)
+ +- ReusedExchange (39)
(1) Scan parquet default.store_sales
@@ -62,7 +65,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s
Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3))
-(4) ReusedExchange [Reuses operator id: 51]
+(4) ReusedExchange [Reuses operator id: 54]
Output [1]: [d_date_sk#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -140,210 +143,222 @@ Join condition: None
Output [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_sk#14, i_item_id#15, i_item_desc#16]
-(22) Sort [codegen id : 7]
+(22) Exchange
+Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
+Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [id=#18]
+
+(23) Sort [codegen id : 8]
Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
Arguments: [ss_customer_sk#2 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0
-(23) Scan parquet default.store_returns
-Output [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22]
+(24) Scan parquet default.store_returns
+Output [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(sr_returned_date_sk#22), dynamicpruningexpression(sr_returned_date_sk#22 IN dynamicpruning#23)]
+PartitionFilters: [isnotnull(sr_returned_date_sk#23), dynamicpruningexpression(sr_returned_date_sk#23 IN dynamicpruning#24)]
PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int,sr_return_quantity:int>
-(24) ColumnarToRow [codegen id : 9]
-Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22]
+(25) ColumnarToRow [codegen id : 10]
+Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23]
-(25) Filter [codegen id : 9]
-Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22]
-Condition : ((isnotnull(sr_customer_sk#19) AND isnotnull(sr_item_sk#18)) AND isnotnull(sr_ticket_number#20))
+(26) Filter [codegen id : 10]
+Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23]
+Condition : ((isnotnull(sr_customer_sk#20) AND isnotnull(sr_item_sk#19)) AND isnotnull(sr_ticket_number#21))
-(26) ReusedExchange [Reuses operator id: 56]
-Output [1]: [d_date_sk#24]
+(27) ReusedExchange [Reuses operator id: 59]
+Output [1]: [d_date_sk#25]
-(27) BroadcastHashJoin [codegen id : 9]
-Left keys [1]: [sr_returned_date_sk#22]
-Right keys [1]: [d_date_sk#24]
+(28) BroadcastHashJoin [codegen id : 10]
+Left keys [1]: [sr_returned_date_sk#23]
+Right keys [1]: [d_date_sk#25]
Join condition: None
-(28) Project [codegen id : 9]
-Output [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
-Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22, d_date_sk#24]
+(29) Project [codegen id : 10]
+Output [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22]
+Input [6]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23, d_date_sk#25]
-(29) Exchange
-Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
-Arguments: hashpartitioning(sr_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#25]
+(30) Exchange
+Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22]
+Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21, 5), ENSURE_REQUIREMENTS, [id=#26]
-(30) Sort [codegen id : 10]
-Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
-Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST, sr_ticket_number#20 ASC NULLS FIRST], false, 0
+(31) Sort [codegen id : 11]
+Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22]
+Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST, sr_ticket_number#21 ASC NULLS FIRST], false, 0
-(31) SortMergeJoin [codegen id : 11]
+(32) SortMergeJoin [codegen id : 12]
Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4]
-Right keys [3]: [sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20]
+Right keys [3]: [sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21]
Join condition: None
-(32) Project [codegen id : 11]
-Output [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21]
-Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21]
+(33) Project [codegen id : 12]
+Output [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22]
+Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22]
+
+(34) Exchange
+Input [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22]
+Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#27]
-(33) Scan parquet default.catalog_sales
-Output [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29]
+(35) Sort [codegen id : 13]
+Input [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22]
+Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST], false, 0
+
+(36) Scan parquet default.catalog_sales
+Output [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(cs_sold_date_sk#29), dynamicpruningexpression(cs_sold_date_sk#29 IN dynamicpruning#30)]
+PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)]
PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_quantity:int>
-(34) ColumnarToRow [codegen id : 13]
-Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29]
+(37) ColumnarToRow [codegen id : 15]
+Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31]
-(35) Filter [codegen id : 13]
-Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29]
-Condition : (isnotnull(cs_bill_customer_sk#26) AND isnotnull(cs_item_sk#27))
+(38) Filter [codegen id : 15]
+Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31]
+Condition : (isnotnull(cs_bill_customer_sk#28) AND isnotnull(cs_item_sk#29))
-(36) ReusedExchange [Reuses operator id: 61]
-Output [1]: [d_date_sk#31]
+(39) ReusedExchange [Reuses operator id: 64]
+Output [1]: [d_date_sk#33]
-(37) BroadcastHashJoin [codegen id : 13]
-Left keys [1]: [cs_sold_date_sk#29]
-Right keys [1]: [d_date_sk#31]
+(40) BroadcastHashJoin [codegen id : 15]
+Left keys [1]: [cs_sold_date_sk#31]
+Right keys [1]: [d_date_sk#33]
Join condition: None
-(38) Project [codegen id : 13]
-Output [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28]
-Input [5]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29, d_date_sk#31]
+(41) Project [codegen id : 15]
+Output [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30]
+Input [5]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31, d_date_sk#33]
-(39) Exchange
-Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28]
-Arguments: hashpartitioning(cs_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#32]
+(42) Exchange
+Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30]
+Arguments: hashpartitioning(cs_bill_customer_sk#28, cs_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#34]
-(40) Sort [codegen id : 14]
-Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28]
-Arguments: [cs_bill_customer_sk#26 ASC NULLS FIRST, cs_item_sk#27 ASC NULLS FIRST], false, 0
+(43) Sort [codegen id : 16]
+Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30]
+Arguments: [cs_bill_customer_sk#28 ASC NULLS FIRST, cs_item_sk#29 ASC NULLS FIRST], false, 0
-(41) SortMergeJoin [codegen id : 15]
-Left keys [2]: [sr_customer_sk#19, sr_item_sk#18]
-Right keys [2]: [cs_bill_customer_sk#26, cs_item_sk#27]
+(44) SortMergeJoin [codegen id : 17]
+Left keys [2]: [sr_customer_sk#20, sr_item_sk#19]
+Right keys [2]: [cs_bill_customer_sk#28, cs_item_sk#29]
Join condition: None
-(42) Project [codegen id : 15]
-Output [7]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
-Input [11]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21, cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28]
+(45) Project [codegen id : 17]
+Output [7]: [ss_quantity#5, sr_return_quantity#22, cs_quantity#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
+Input [11]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22, cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30]
-(43) HashAggregate [codegen id : 15]
-Input [7]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
+(46) HashAggregate [codegen id : 17]
+Input [7]: [ss_quantity#5, sr_return_quantity#22, cs_quantity#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16]
Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11]
-Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#21), partial_sum(cs_quantity#28)]
-Aggregate Attributes [3]: [sum#33, sum#34, sum#35]
-Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#36, sum#37, sum#38]
+Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#22), partial_sum(cs_quantity#30)]
+Aggregate Attributes [3]: [sum#35, sum#36, sum#37]
+Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#38, sum#39, sum#40]
-(44) Exchange
-Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#36, sum#37, sum#38]
-Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#39]
+(47) Exchange
+Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#38, sum#39, sum#40]
+Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#41]
-(45) HashAggregate [codegen id : 16]
-Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#36, sum#37, sum#38]
+(48) HashAggregate [codegen id : 18]
+Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#38, sum#39, sum#40]
Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11]
-Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#21), sum(cs_quantity#28)]
-Aggregate Attributes [3]: [sum(ss_quantity#5)#40, sum(sr_return_quantity#21)#41, sum(cs_quantity#28)#42]
-Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum(ss_quantity#5)#40 AS store_sales_quantity#43, sum(sr_return_quantity#21)#41 AS store_returns_quantity#44, sum(cs_quantity#28)#42 AS catalog_sales_quantity#45]
+Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#22), sum(cs_quantity#30)]
+Aggregate Attributes [3]: [sum(ss_quantity#5)#42, sum(sr_return_quantity#22)#43, sum(cs_quantity#30)#44]
+Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum(ss_quantity#5)#42 AS store_sales_quantity#45, sum(sr_return_quantity#22)#43 AS store_returns_quantity#46, sum(cs_quantity#30)#44 AS catalog_sales_quantity#47]
-(46) TakeOrderedAndProject
-Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#43, store_returns_quantity#44, catalog_sales_quantity#45]
-Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#43, store_returns_quantity#44, catalog_sales_quantity#45]
+(49) TakeOrderedAndProject
+Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47]
+Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
-BroadcastExchange (51)
-+- * Project (50)
- +- * Filter (49)
- +- * ColumnarToRow (48)
- +- Scan parquet default.date_dim (47)
+BroadcastExchange (54)
++- * Project (53)
+ +- * Filter (52)
+ +- * ColumnarToRow (51)
+ +- Scan parquet default.date_dim (50)
-(47) Scan parquet default.date_dim
-Output [3]: [d_date_sk#8, d_year#46, d_moy#47]
+(50) Scan parquet default.date_dim
+Output [3]: [d_date_sk#8, d_year#48, d_moy#49]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(48) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#8, d_year#46, d_moy#47]
+(51) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#8, d_year#48, d_moy#49]
-(49) Filter [codegen id : 1]
-Input [3]: [d_date_sk#8, d_year#46, d_moy#47]
-Condition : ((((isnotnull(d_moy#47) AND isnotnull(d_year#46)) AND (d_moy#47 = 9)) AND (d_year#46 = 1999)) AND isnotnull(d_date_sk#8))
+(52) Filter [codegen id : 1]
+Input [3]: [d_date_sk#8, d_year#48, d_moy#49]
+Condition : ((((isnotnull(d_moy#49) AND isnotnull(d_year#48)) AND (d_moy#49 = 9)) AND (d_year#48 = 1999)) AND isnotnull(d_date_sk#8))
-(50) Project [codegen id : 1]
+(53) Project [codegen id : 1]
Output [1]: [d_date_sk#8]
-Input [3]: [d_date_sk#8, d_year#46, d_moy#47]
+Input [3]: [d_date_sk#8, d_year#48, d_moy#49]
-(51) BroadcastExchange
+(54) BroadcastExchange
Input [1]: [d_date_sk#8]
-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=#50]
-Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23
-BroadcastExchange (56)
-+- * Project (55)
- +- * Filter (54)
- +- * ColumnarToRow (53)
- +- Scan parquet default.date_dim (52)
+Subquery:2 Hosting operator id = 24 Hosting Expression = sr_returned_date_sk#23 IN dynamicpruning#24
+BroadcastExchange (59)
++- * Project (58)
+ +- * Filter (57)
+ +- * ColumnarToRow (56)
+ +- Scan parquet default.date_dim (55)
-(52) Scan parquet default.date_dim
-Output [3]: [d_date_sk#24, d_year#49, d_moy#50]
+(55) Scan parquet default.date_dim
+Output [3]: [d_date_sk#25, d_year#51, d_moy#52]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(53) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#24, d_year#49, d_moy#50]
+(56) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#25, d_year#51, d_moy#52]
-(54) Filter [codegen id : 1]
-Input [3]: [d_date_sk#24, d_year#49, d_moy#50]
-Condition : (((((isnotnull(d_moy#50) AND isnotnull(d_year#49)) AND (d_moy#50 >= 9)) AND (d_moy#50 <= 12)) AND (d_year#49 = 1999)) AND isnotnull(d_date_sk#24))
+(57) Filter [codegen id : 1]
+Input [3]: [d_date_sk#25, d_year#51, d_moy#52]
+Condition : (((((isnotnull(d_moy#52) AND isnotnull(d_year#51)) AND (d_moy#52 >= 9)) AND (d_moy#52 <= 12)) AND (d_year#51 = 1999)) AND isnotnull(d_date_sk#25))
-(55) Project [codegen id : 1]
-Output [1]: [d_date_sk#24]
-Input [3]: [d_date_sk#24, d_year#49, d_moy#50]
+(58) Project [codegen id : 1]
+Output [1]: [d_date_sk#25]
+Input [3]: [d_date_sk#25, d_year#51, d_moy#52]
-(56) BroadcastExchange
-Input [1]: [d_date_sk#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51]
+(59) BroadcastExchange
+Input [1]: [d_date_sk#25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53]
-Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#30
-BroadcastExchange (61)
-+- * Project (60)
- +- * Filter (59)
- +- * ColumnarToRow (58)
- +- Scan parquet default.date_dim (57)
+Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#32
+BroadcastExchange (64)
++- * Project (63)
+ +- * Filter (62)
+ +- * ColumnarToRow (61)
+ +- Scan parquet default.date_dim (60)
-(57) Scan parquet default.date_dim
-Output [2]: [d_date_sk#31, d_year#52]
+(60) Scan parquet default.date_dim
+Output [2]: [d_date_sk#33, d_year#54]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
-(58) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#31, d_year#52]
+(61) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#33, d_year#54]
-(59) Filter [codegen id : 1]
-Input [2]: [d_date_sk#31, d_year#52]
-Condition : (d_year#52 IN (1999,2000,2001) AND isnotnull(d_date_sk#31))
+(62) Filter [codegen id : 1]
+Input [2]: [d_date_sk#33, d_year#54]
+Condition : (d_year#54 IN (1999,2000,2001) AND isnotnull(d_date_sk#33))
-(60) Project [codegen id : 1]
-Output [1]: [d_date_sk#31]
-Input [2]: [d_date_sk#31, d_year#52]
+(63) Project [codegen id : 1]
+Output [1]: [d_date_sk#33]
+Input [2]: [d_date_sk#33, d_year#54]
-(61) BroadcastExchange
-Input [1]: [d_date_sk#31]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53]
+(64) BroadcastExchange
+Input [1]: [d_date_sk#33]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt
index 0db54fe..5463f3f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt
@@ -1,90 +1,97 @@
TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity]
- WholeStageCodegen (16)
+ WholeStageCodegen (18)
HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum]
InputAdapter
Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1
- WholeStageCodegen (15)
+ WholeStageCodegen (17)
HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum]
Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc]
SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk]
InputAdapter
- WholeStageCodegen (11)
- Project [ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity]
- SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
- InputAdapter
- WholeStageCodegen (7)
- Sort [ss_customer_sk,ss_item_sk,ss_ticket_number]
- Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc]
- SortMergeJoin [ss_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [ss_item_sk]
- InputAdapter
- Exchange [ss_item_sk] #2
- WholeStageCodegen (3)
- Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #3
- WholeStageCodegen (1)
- Project [d_date_sk]
- Filter [d_moy,d_year,d_date_sk]
+ WholeStageCodegen (13)
+ Sort [sr_customer_sk,sr_item_sk]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_item_sk] #2
+ WholeStageCodegen (12)
+ Project [ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity]
+ SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ WholeStageCodegen (8)
+ Sort [ss_customer_sk,ss_item_sk,ss_ticket_number]
+ InputAdapter
+ Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3
+ WholeStageCodegen (7)
+ Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc]
+ SortMergeJoin [ss_item_sk,i_item_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_item_sk] #4
+ WholeStageCodegen (3)
+ Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_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]
+ InputAdapter
+ ReusedExchange [d_date_sk] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [s_store_sk]
ColumnarToRow
InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
+ Scan parquet default.store [s_store_sk,s_store_id,s_store_name]
+ InputAdapter
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
InputAdapter
- ReusedExchange [d_date_sk] #3
- InputAdapter
- BroadcastExchange #4
- WholeStageCodegen (2)
- Filter [s_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store [s_store_sk,s_store_id,s_store_name]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #5
- WholeStageCodegen (5)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_item_id,i_item_desc]
- InputAdapter
- WholeStageCodegen (10)
- Sort [sr_customer_sk,sr_item_sk,sr_ticket_number]
- InputAdapter
- Exchange [sr_item_sk] #6
- WholeStageCodegen (9)
- Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity]
- BroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- Filter [sr_customer_sk,sr_item_sk,sr_ticket_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
- SubqueryBroadcast [d_date_sk] #2
- BroadcastExchange #7
- 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]
- InputAdapter
- ReusedExchange [d_date_sk] #7
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_item_id,i_item_desc]
+ InputAdapter
+ WholeStageCodegen (11)
+ Sort [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8
+ WholeStageCodegen (10)
+ Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity]
+ BroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ Filter [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #9
+ 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]
+ InputAdapter
+ ReusedExchange [d_date_sk] #9
InputAdapter
- WholeStageCodegen (14)
+ WholeStageCodegen (16)
Sort [cs_bill_customer_sk,cs_item_sk]
InputAdapter
- Exchange [cs_item_sk] #8
- WholeStageCodegen (13)
+ Exchange [cs_bill_customer_sk,cs_item_sk] #10
+ WholeStageCodegen (15)
Project [cs_bill_customer_sk,cs_item_sk,cs_quantity]
BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
Filter [cs_bill_customer_sk,cs_item_sk]
@@ -92,7 +99,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
InputAdapter
Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
- BroadcastExchange #9
+ BroadcastExchange #11
WholeStageCodegen (1)
Project [d_date_sk]
Filter [d_year,d_date_sk]
@@ -100,4 +107,4 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
InputAdapter
Scan parquet default.date_dim [d_date_sk,d_year]
InputAdapter
- ReusedExchange [d_date_sk] #9
+ ReusedExchange [d_date_sk] #11
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
index 44a9564..529b9c8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
@@ -1,53 +1,56 @@
== Physical Plan ==
-TakeOrderedAndProject (49)
-+- * Project (48)
- +- * SortMergeJoin Inner (47)
- :- * Project (41)
- : +- * SortMergeJoin Inner (40)
- : :- * Sort (32)
- : : +- * Project (31)
- : : +- * Filter (30)
- : : +- Window (29)
- : : +- * Filter (28)
- : : +- Window (27)
- : : +- * Sort (26)
- : : +- Exchange (25)
- : : +- * HashAggregate (24)
- : : +- Exchange (23)
- : : +- * HashAggregate (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.store_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.store (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (39)
- : +- * Project (38)
- : +- Window (37)
- : +- * Sort (36)
- : +- Exchange (35)
- : +- * HashAggregate (34)
- : +- ReusedExchange (33)
- +- * Sort (46)
- +- * Project (45)
- +- Window (44)
- +- * Sort (43)
- +- ReusedExchange (42)
+TakeOrderedAndProject (52)
++- * Project (51)
+ +- * SortMergeJoin Inner (50)
+ :- * Project (43)
+ : +- * SortMergeJoin Inner (42)
+ : :- * Sort (33)
+ : : +- Exchange (32)
+ : : +- * Project (31)
+ : : +- * Filter (30)
+ : : +- Window (29)
+ : : +- * Filter (28)
+ : : +- Window (27)
+ : : +- * Sort (26)
+ : : +- Exchange (25)
+ : : +- * HashAggregate (24)
+ : : +- Exchange (23)
+ : : +- * HashAggregate (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.store_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.store (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (41)
+ : +- Exchange (40)
+ : +- * Project (39)
+ : +- Window (38)
+ : +- * Sort (37)
+ : +- Exchange (36)
+ : +- * HashAggregate (35)
+ : +- ReusedExchange (34)
+ +- * Sort (49)
+ +- Exchange (48)
+ +- * Project (47)
+ +- Window (46)
+ +- * Sort (45)
+ +- ReusedExchange (44)
(1) Scan parquet default.store_sales
@@ -65,7 +68,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2))
-(4) ReusedExchange [Reuses operator id: 53]
+(4) ReusedExchange [Reuses operator id: 56]
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -189,106 +192,118 @@ Condition : ((isnotnull(avg_monthly_sales#26) AND (avg_monthly_sales#26 > 0.0000
Output [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25]
Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, _w0#23, rn#25, avg_monthly_sales#26]
-(32) Sort [codegen id : 11]
+(32) Exchange
+Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25]
+Arguments: hashpartitioning(i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25, 5), ENSURE_REQUIREMENTS, [id=#27]
+
+(33) Sort [codegen id : 12]
Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25]
Arguments: [i_category#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST, s_company_name#11 ASC NULLS FIRST, rn#25 ASC NULLS FIRST], false, 0
-(33) ReusedExchange [Reuses operator id: 23]
-Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33]
+(34) ReusedExchange [Reuses operator id: 23]
+Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34]
-(34) HashAggregate [codegen id : 19]
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33]
-Keys [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32]
-Functions [1]: [sum(UnscaledValue(ss_sales_price#34))]
-Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#34))#21]
-Results [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#34))#21,17,2) AS sum_sales#22]
+(35) HashAggregate [codegen id : 20]
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34]
+Keys [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33]
+Functions [1]: [sum(UnscaledValue(ss_sales_price#35))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#21]
+Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#21,17,2) AS sum_sales#22]
-(35) Exchange
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22]
-Arguments: hashpartitioning(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, 5), ENSURE_REQUIREMENTS, [id=#35]
+(36) Exchange
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22]
+Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#36]
-(36) Sort [codegen id : 20]
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22]
-Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST], false, 0
+(37) Sort [codegen id : 21]
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22]
+Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0
-(37) Window
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22]
-Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#27, i_brand#28, s_store_name#29, s_company_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST]
+(38) Window
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22]
+Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST]
-(38) Project [codegen id : 21]
-Output [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#22 AS sum_sales#37, rn#36]
-Input [8]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22, rn#36]
+(39) Project [codegen id : 22]
+Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#22 AS sum_sales#38, rn#37]
+Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22, rn#37]
-(39) Sort [codegen id : 21]
-Input [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36]
-Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, (rn#36 + 1) ASC NULLS FIRST], false, 0
+(40) Exchange
+Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37]
+Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#39]
-(40) SortMergeJoin [codegen id : 22]
+(41) Sort [codegen id : 23]
+Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37]
+Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0
+
+(42) SortMergeJoin [codegen id : 24]
Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25]
-Right keys [5]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, (rn#36 + 1)]
+Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1)]
Join condition: None
-(41) Project [codegen id : 22]
-Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37]
-Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36]
+(43) Project [codegen id : 24]
+Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38]
+Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37]
+
+(44) ReusedExchange [Reuses operator id: 36]
+Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22]
-(42) ReusedExchange [Reuses operator id: 35]
-Output [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22]
+(45) Sort [codegen id : 33]
+Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22]
+Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0
-(43) Sort [codegen id : 31]
-Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22]
-Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0
+(46) Window
+Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22]
+Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST]
-(44) Window
-Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22]
-Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, s_store_name#40, s_company_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST]
+(47) Project [codegen id : 34]
+Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#22 AS sum_sales#47, rn#46]
+Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22, rn#46]
-(45) Project [codegen id : 32]
-Output [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#22 AS sum_sales#45, rn#44]
-Input [8]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22, rn#44]
+(48) Exchange
+Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46]
+Arguments: hashpartitioning(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#48]
-(46) Sort [codegen id : 32]
-Input [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44]
-Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, (rn#44 - 1) ASC NULLS FIRST], false, 0
+(49) Sort [codegen id : 35]
+Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46]
+Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0
-(47) SortMergeJoin [codegen id : 33]
+(50) SortMergeJoin [codegen id : 36]
Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25]
-Right keys [5]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, (rn#44 - 1)]
+Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1)]
Join condition: None
-(48) Project [codegen id : 33]
-Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#46, sum_sales#45 AS nsum#47]
-Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44]
+(51) Project [codegen id : 36]
+Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#38 AS psum#49, sum_sales#47 AS nsum#50]
+Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46]
-(49) TakeOrderedAndProject
-Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47]
+(52) TakeOrderedAndProject
+Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
+Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (53)
-+- * Filter (52)
- +- * ColumnarToRow (51)
- +- Scan parquet default.date_dim (50)
+BroadcastExchange (56)
++- * Filter (55)
+ +- * ColumnarToRow (54)
+ +- Scan parquet default.date_dim (53)
-(50) Scan parquet default.date_dim
+(53) Scan parquet default.date_dim
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(51) ColumnarToRow [codegen id : 1]
+(54) ColumnarToRow [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-(52) Filter [codegen id : 1]
+(55) Filter [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6))
-(53) BroadcastExchange
+(56) BroadcastExchange
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt
index aa2346c..07c75d9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt
@@ -1,95 +1,104 @@
TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum]
- WholeStageCodegen (33)
+ WholeStageCodegen (36)
Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales]
SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn]
InputAdapter
- WholeStageCodegen (22)
+ WholeStageCodegen (24)
Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales]
SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn]
InputAdapter
- WholeStageCodegen (11)
+ WholeStageCodegen (12)
Sort [i_category,i_brand,s_store_name,s_company_name,rn]
- Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
- Filter [avg_monthly_sales,sum_sales]
- InputAdapter
- Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year]
- WholeStageCodegen (10)
- Filter [d_year]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
- WholeStageCodegen (9)
- Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,s_store_name,s_company_name] #1
- WholeStageCodegen (8)
- HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum]
- InputAdapter
- Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2
- WholeStageCodegen (7)
- HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum]
- Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- SortMergeJoin [ss_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [ss_item_sk]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #1
+ WholeStageCodegen (11)
+ Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
+ Filter [avg_monthly_sales,sum_sales]
+ InputAdapter
+ Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year]
+ WholeStageCodegen (10)
+ Filter [d_year]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
+ WholeStageCodegen (9)
+ Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name] #2
+ WholeStageCodegen (8)
+ HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3
+ WholeStageCodegen (7)
+ HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum]
+ Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
+ SortMergeJoin [ss_item_sk,i_item_sk]
InputAdapter
- Exchange [ss_item_sk] #3
- WholeStageCodegen (3)
- Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Filter [ss_item_sk,ss_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #4
- WholeStageCodegen (1)
- Filter [d_year,d_moy,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
- InputAdapter
- ReusedExchange [d_date_sk,d_year,d_moy] #4
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (2)
- Filter [s_store_sk,s_store_name,s_company_name]
- ColumnarToRow
+ WholeStageCodegen (4)
+ Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_item_sk] #4
+ WholeStageCodegen (3)
+ Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Filter [ss_item_sk,ss_store_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ Filter [d_year,d_moy,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
- Scan parquet default.store [s_store_sk,s_store_name,s_company_name]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
+ ReusedExchange [d_date_sk,d_year,d_moy] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [s_store_sk,s_store_name,s_company_name]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
- Exchange [i_item_sk] #6
- WholeStageCodegen (5)
- Filter [i_item_sk,i_category,i_brand]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_brand,i_category]
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk,i_category,i_brand]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_brand,i_category]
InputAdapter
- WholeStageCodegen (21)
+ WholeStageCodegen (23)
Sort [i_category,i_brand,s_store_name,s_company_name,rn]
- Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
- WholeStageCodegen (20)
- Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,s_store_name,s_company_name] #7
- WholeStageCodegen (19)
- HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum]
- InputAdapter
- ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8
+ WholeStageCodegen (22)
+ Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
+ WholeStageCodegen (21)
+ Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name] #9
+ WholeStageCodegen (20)
+ HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3
InputAdapter
- WholeStageCodegen (32)
+ WholeStageCodegen (35)
Sort [i_category,i_brand,s_store_name,s_company_name,rn]
- Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
- WholeStageCodegen (31)
- Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
- InputAdapter
- ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10
+ WholeStageCodegen (34)
+ Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
+ WholeStageCodegen (33)
+ Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #9
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
index ad356d4..ef8d64c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
@@ -1,53 +1,56 @@
== Physical Plan ==
-TakeOrderedAndProject (49)
-+- * Project (48)
- +- * SortMergeJoin Inner (47)
- :- * Project (41)
- : +- * SortMergeJoin Inner (40)
- : :- * Sort (32)
- : : +- * Project (31)
- : : +- * Filter (30)
- : : +- Window (29)
- : : +- * Filter (28)
- : : +- Window (27)
- : : +- * Sort (26)
- : : +- Exchange (25)
- : : +- * HashAggregate (24)
- : : +- Exchange (23)
- : : +- * HashAggregate (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.catalog_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.call_center (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (39)
- : +- * Project (38)
- : +- Window (37)
- : +- * Sort (36)
- : +- Exchange (35)
- : +- * HashAggregate (34)
- : +- ReusedExchange (33)
- +- * Sort (46)
- +- * Project (45)
- +- Window (44)
- +- * Sort (43)
- +- ReusedExchange (42)
+TakeOrderedAndProject (52)
++- * Project (51)
+ +- * SortMergeJoin Inner (50)
+ :- * Project (43)
+ : +- * SortMergeJoin Inner (42)
+ : :- * Sort (33)
+ : : +- Exchange (32)
+ : : +- * Project (31)
+ : : +- * Filter (30)
+ : : +- Window (29)
+ : : +- * Filter (28)
+ : : +- Window (27)
+ : : +- * Sort (26)
+ : : +- Exchange (25)
+ : : +- * HashAggregate (24)
+ : : +- Exchange (23)
+ : : +- * HashAggregate (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.catalog_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.call_center (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (41)
+ : +- Exchange (40)
+ : +- * Project (39)
+ : +- Window (38)
+ : +- * Sort (37)
+ : +- Exchange (36)
+ : +- * HashAggregate (35)
+ : +- ReusedExchange (34)
+ +- * Sort (49)
+ +- Exchange (48)
+ +- * Project (47)
+ +- Window (46)
+ +- * Sort (45)
+ +- ReusedExchange (44)
(1) Scan parquet default.catalog_sales
@@ -65,7 +68,7 @@ Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk
Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk#4]
Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_call_center_sk#1))
-(4) ReusedExchange [Reuses operator id: 53]
+(4) ReusedExchange [Reuses operator id: 56]
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -189,106 +192,118 @@ Condition : ((isnotnull(avg_monthly_sales#25) AND (avg_monthly_sales#25 > 0.0000
Output [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24]
Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, _w0#22, rn#24, avg_monthly_sales#25]
-(32) Sort [codegen id : 11]
+(32) Exchange
+Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24]
+Arguments: hashpartitioning(i_category#15, i_brand#14, cc_name#10, rn#24, 5), ENSURE_REQUIREMENTS, [id=#26]
+
+(33) Sort [codegen id : 12]
Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24]
Arguments: [i_category#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST, rn#24 ASC NULLS FIRST], false, 0
-(33) ReusedExchange [Reuses operator id: 23]
-Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31]
+(34) ReusedExchange [Reuses operator id: 23]
+Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32]
-(34) HashAggregate [codegen id : 19]
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31]
-Keys [5]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30]
-Functions [1]: [sum(UnscaledValue(cs_sales_price#32))]
-Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#32))#20]
-Results [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#32))#20,17,2) AS sum_sales#21]
+(35) HashAggregate [codegen id : 20]
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32]
+Keys [5]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31]
+Functions [1]: [sum(UnscaledValue(cs_sales_price#33))]
+Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#20]
+Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#20,17,2) AS sum_sales#21]
-(35) Exchange
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21]
-Arguments: hashpartitioning(i_category#26, i_brand#27, cc_name#28, 5), ENSURE_REQUIREMENTS, [id=#33]
+(36) Exchange
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21]
+Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#34]
-(36) Sort [codegen id : 20]
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21]
-Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0
+(37) Sort [codegen id : 21]
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21]
+Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0
-(37) Window
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21]
-Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST]
+(38) Window
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21]
+Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST]
-(38) Project [codegen id : 21]
-Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#21 AS sum_sales#35, rn#34]
-Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21, rn#34]
+(39) Project [codegen id : 22]
+Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#21 AS sum_sales#36, rn#35]
+Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35]
-(39) Sort [codegen id : 21]
-Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34]
-Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, (rn#34 + 1) ASC NULLS FIRST], false, 0
+(40) Exchange
+Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35]
+Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#35 + 1), 5), ENSURE_REQUIREMENTS, [id=#37]
-(40) SortMergeJoin [codegen id : 22]
+(41) Sort [codegen id : 23]
+Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35]
+Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#35 + 1) ASC NULLS FIRST], false, 0
+
+(42) SortMergeJoin [codegen id : 24]
Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24]
-Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#34 + 1)]
+Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#35 + 1)]
Join condition: None
-(41) Project [codegen id : 22]
-Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35]
-Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34]
+(43) Project [codegen id : 24]
+Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36]
+Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35]
+
+(44) ReusedExchange [Reuses operator id: 36]
+Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21]
-(42) ReusedExchange [Reuses operator id: 35]
-Output [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21]
+(45) Sort [codegen id : 33]
+Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21]
+Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0
-(43) Sort [codegen id : 31]
-Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21]
-Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST], false, 0
+(46) Window
+Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21]
+Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST]
-(44) Window
-Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21]
-Arguments: [rank(d_year#39, d_moy#40) windowspecdefinition(i_category#36, i_brand#37, cc_name#38, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#36, i_brand#37, cc_name#38], [d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST]
+(47) Project [codegen id : 34]
+Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#21 AS sum_sales#44, rn#43]
+Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21, rn#43]
-(45) Project [codegen id : 32]
-Output [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#21 AS sum_sales#42, rn#41]
-Input [7]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21, rn#41]
+(48) Exchange
+Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43]
+Arguments: hashpartitioning(i_category#38, i_brand#39, cc_name#40, (rn#43 - 1), 5), ENSURE_REQUIREMENTS, [id=#45]
-(46) Sort [codegen id : 32]
-Input [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41]
-Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, (rn#41 - 1) ASC NULLS FIRST], false, 0
+(49) Sort [codegen id : 35]
+Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43]
+Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, (rn#43 - 1) ASC NULLS FIRST], false, 0
-(47) SortMergeJoin [codegen id : 33]
+(50) SortMergeJoin [codegen id : 36]
Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24]
-Right keys [4]: [i_category#36, i_brand#37, cc_name#38, (rn#41 - 1)]
+Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#43 - 1)]
Join condition: None
-(48) Project [codegen id : 33]
-Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#43, sum_sales#42 AS nsum#44]
-Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41]
+(51) Project [codegen id : 36]
+Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#46, sum_sales#44 AS nsum#47]
+Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43]
-(49) TakeOrderedAndProject
-Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44]
+(52) TakeOrderedAndProject
+Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
+Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (53)
-+- * Filter (52)
- +- * ColumnarToRow (51)
- +- Scan parquet default.date_dim (50)
+BroadcastExchange (56)
++- * Filter (55)
+ +- * ColumnarToRow (54)
+ +- Scan parquet default.date_dim (53)
-(50) Scan parquet default.date_dim
+(53) Scan parquet default.date_dim
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(51) ColumnarToRow [codegen id : 1]
+(54) ColumnarToRow [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-(52) Filter [codegen id : 1]
+(55) Filter [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6))
-(53) BroadcastExchange
+(56) BroadcastExchange
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt
index b488806..3bf10f8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt
@@ -1,95 +1,104 @@
TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum]
- WholeStageCodegen (33)
+ WholeStageCodegen (36)
Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales]
SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn]
InputAdapter
- WholeStageCodegen (22)
+ WholeStageCodegen (24)
Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales]
SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn]
InputAdapter
- WholeStageCodegen (11)
+ WholeStageCodegen (12)
Sort [i_category,i_brand,cc_name,rn]
- Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
- Filter [avg_monthly_sales,sum_sales]
- InputAdapter
- Window [_w0,i_category,i_brand,cc_name,d_year]
- WholeStageCodegen (10)
- Filter [d_year]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,cc_name]
- WholeStageCodegen (9)
- Sort [i_category,i_brand,cc_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,cc_name] #1
- WholeStageCodegen (8)
- HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum]
- InputAdapter
- Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2
- WholeStageCodegen (7)
- HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum]
- Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- SortMergeJoin [cs_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [cs_item_sk]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,rn] #1
+ WholeStageCodegen (11)
+ Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
+ Filter [avg_monthly_sales,sum_sales]
+ InputAdapter
+ Window [_w0,i_category,i_brand,cc_name,d_year]
+ WholeStageCodegen (10)
+ Filter [d_year]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,cc_name]
+ WholeStageCodegen (9)
+ Sort [i_category,i_brand,cc_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name] #2
+ WholeStageCodegen (8)
+ HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3
+ WholeStageCodegen (7)
+ HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum]
+ Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
+ SortMergeJoin [cs_item_sk,i_item_sk]
InputAdapter
- Exchange [cs_item_sk] #3
- WholeStageCodegen (3)
- Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name]
- BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
- Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy]
- BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- Filter [cs_item_sk,cs_call_center_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #4
- WholeStageCodegen (1)
- Filter [d_year,d_moy,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
- InputAdapter
- ReusedExchange [d_date_sk,d_year,d_moy] #4
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (2)
- Filter [cc_call_center_sk,cc_name]
- ColumnarToRow
+ WholeStageCodegen (4)
+ Sort [cs_item_sk]
+ InputAdapter
+ Exchange [cs_item_sk] #4
+ WholeStageCodegen (3)
+ Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name]
+ BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ Filter [cs_item_sk,cs_call_center_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ Filter [d_year,d_moy,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
- Scan parquet default.call_center [cc_call_center_sk,cc_name]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
+ ReusedExchange [d_date_sk,d_year,d_moy] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [cc_call_center_sk,cc_name]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.call_center [cc_call_center_sk,cc_name]
InputAdapter
- Exchange [i_item_sk] #6
- WholeStageCodegen (5)
- Filter [i_item_sk,i_category,i_brand]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_brand,i_category]
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk,i_category,i_brand]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_brand,i_category]
InputAdapter
- WholeStageCodegen (21)
+ WholeStageCodegen (23)
Sort [i_category,i_brand,cc_name,rn]
- Project [i_category,i_brand,cc_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,cc_name]
- WholeStageCodegen (20)
- Sort [i_category,i_brand,cc_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,cc_name] #7
- WholeStageCodegen (19)
- HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum]
- InputAdapter
- ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,rn] #8
+ WholeStageCodegen (22)
+ Project [i_category,i_brand,cc_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,cc_name]
+ WholeStageCodegen (21)
+ Sort [i_category,i_brand,cc_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name] #9
+ WholeStageCodegen (20)
+ HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3
InputAdapter
- WholeStageCodegen (32)
+ WholeStageCodegen (35)
Sort [i_category,i_brand,cc_name,rn]
- Project [i_category,i_brand,cc_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,cc_name]
- WholeStageCodegen (31)
- Sort [i_category,i_brand,cc_name,d_year,d_moy]
- InputAdapter
- ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,rn] #10
+ WholeStageCodegen (34)
+ Project [i_category,i_brand,cc_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,cc_name]
+ WholeStageCodegen (33)
+ Sort [i_category,i_brand,cc_name,d_year,d_moy]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #9
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt
index d4ecd7a..c6971f3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt
@@ -1,72 +1,74 @@
== Physical Plan ==
-TakeOrderedAndProject (68)
-+- * HashAggregate (67)
- +- Exchange (66)
- +- * HashAggregate (65)
- +- * Project (64)
- +- * SortMergeJoin LeftOuter (63)
- :- * Sort (56)
- : +- * Project (55)
- : +- * BroadcastHashJoin LeftOuter BuildRight (54)
- : :- * Project (49)
- : : +- * SortMergeJoin Inner (48)
- : : :- * Sort (36)
- : : : +- * Project (35)
- : : : +- * BroadcastHashJoin Inner BuildRight (34)
- : : : :- * Project (32)
- : : : : +- * SortMergeJoin Inner (31)
- : : : : :- * Sort (25)
- : : : : : +- Exchange (24)
- : : : : : +- * Project (23)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (22)
- : : : : : :- * Project (17)
- : : : : : : +- * BroadcastHashJoin Inner BuildRight (16)
- : : : : : : :- * Project (10)
- : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9)
- : : : : : : : :- * Filter (3)
- : : : : : : : : +- * ColumnarToRow (2)
- : : : : : : : : +- Scan parquet default.catalog_sales (1)
- : : : : : : : +- BroadcastExchange (8)
- : : : : : : : +- * Project (7)
- : : : : : : : +- * Filter (6)
- : : : : : : : +- * ColumnarToRow (5)
- : : : : : : : +- Scan parquet default.household_demographics (4)
- : : : : : : +- BroadcastExchange (15)
- : : : : : : +- * Project (14)
- : : : : : : +- * Filter (13)
- : : : : : : +- * ColumnarToRow (12)
- : : : : : : +- Scan parquet default.customer_demographics (11)
- : : : : : +- BroadcastExchange (21)
- : : : : : +- * Filter (20)
- : : : : : +- * ColumnarToRow (19)
- : : : : : +- Scan parquet default.date_dim (18)
- : : : : +- * Sort (30)
- : : : : +- Exchange (29)
- : : : : +- * Filter (28)
- : : : : +- * ColumnarToRow (27)
- : : : : +- Scan parquet default.item (26)
- : : : +- ReusedExchange (33)
- : : +- * Sort (47)
- : : +- Exchange (46)
- : : +- * Project (45)
- : : +- * BroadcastHashJoin Inner BuildRight (44)
- : : :- * Filter (39)
- : : : +- * ColumnarToRow (38)
- : : : +- Scan parquet default.inventory (37)
- : : +- BroadcastExchange (43)
- : : +- * Filter (42)
- : : +- * ColumnarToRow (41)
- : : +- Scan parquet default.warehouse (40)
- : +- BroadcastExchange (53)
- : +- * Filter (52)
- : +- * ColumnarToRow (51)
- : +- Scan parquet default.promotion (50)
- +- * Sort (62)
- +- Exchange (61)
- +- * Project (60)
- +- * Filter (59)
- +- * ColumnarToRow (58)
- +- Scan parquet default.catalog_returns (57)
+TakeOrderedAndProject (70)
++- * HashAggregate (69)
+ +- Exchange (68)
+ +- * HashAggregate (67)
+ +- * Project (66)
+ +- * SortMergeJoin LeftOuter (65)
+ :- * Sort (58)
+ : +- Exchange (57)
+ : +- * Project (56)
+ : +- * BroadcastHashJoin LeftOuter BuildRight (55)
+ : :- * Project (50)
+ : : +- * SortMergeJoin Inner (49)
+ : : :- * Sort (37)
+ : : : +- Exchange (36)
+ : : : +- * Project (35)
+ : : : +- * BroadcastHashJoin Inner BuildRight (34)
+ : : : :- * Project (32)
+ : : : : +- * SortMergeJoin Inner (31)
+ : : : : :- * Sort (25)
+ : : : : : +- Exchange (24)
+ : : : : : +- * Project (23)
+ : : : : : +- * BroadcastHashJoin Inner BuildRight (22)
+ : : : : : :- * Project (17)
+ : : : : : : +- * BroadcastHashJoin Inner BuildRight (16)
+ : : : : : : :- * Project (10)
+ : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9)
+ : : : : : : : :- * Filter (3)
+ : : : : : : : : +- * ColumnarToRow (2)
+ : : : : : : : : +- Scan parquet default.catalog_sales (1)
+ : : : : : : : +- BroadcastExchange (8)
+ : : : : : : : +- * Project (7)
+ : : : : : : : +- * Filter (6)
+ : : : : : : : +- * ColumnarToRow (5)
+ : : : : : : : +- Scan parquet default.household_demographics (4)
+ : : : : : : +- BroadcastExchange (15)
+ : : : : : : +- * Project (14)
+ : : : : : : +- * Filter (13)
+ : : : : : : +- * ColumnarToRow (12)
+ : : : : : : +- Scan parquet default.customer_demographics (11)
+ : : : : : +- BroadcastExchange (21)
+ : : : : : +- * Filter (20)
+ : : : : : +- * ColumnarToRow (19)
+ : : : : : +- Scan parquet default.date_dim (18)
+ : : : : +- * Sort (30)
+ : : : : +- Exchange (29)
+ : : : : +- * Filter (28)
+ : : : : +- * ColumnarToRow (27)
+ : : : : +- Scan parquet default.item (26)
+ : : : +- ReusedExchange (33)
+ : : +- * Sort (48)
+ : : +- Exchange (47)
+ : : +- * Project (46)
+ : : +- * BroadcastHashJoin Inner BuildRight (45)
+ : : :- * Filter (40)
+ : : : +- * ColumnarToRow (39)
+ : : : +- Scan parquet default.inventory (38)
+ : : +- BroadcastExchange (44)
+ : : +- * Filter (43)
+ : : +- * ColumnarToRow (42)
+ : : +- Scan parquet default.warehouse (41)
+ : +- BroadcastExchange (54)
+ : +- * Filter (53)
+ : +- * ColumnarToRow (52)
+ : +- Scan parquet default.promotion (51)
+ +- * Sort (64)
+ +- Exchange (63)
+ +- * Project (62)
+ +- * Filter (61)
+ +- * ColumnarToRow (60)
+ +- Scan parquet default.catalog_returns (59)
(1) Scan parquet default.catalog_sales
@@ -212,7 +214,7 @@ Join condition: None
Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21]
Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21]
-(33) ReusedExchange [Reuses operator id: 79]
+(33) ReusedExchange [Reuses operator id: 81]
Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
(34) BroadcastHashJoin [codegen id : 10]
@@ -224,220 +226,228 @@ Join condition: (d_date#17 > date_add(d_date#24, 5))
Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26]
Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
-(36) Sort [codegen id : 10]
+(36) Exchange
+Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26]
+Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#26, 5), ENSURE_REQUIREMENTS, [id=#27]
+
+(37) Sort [codegen id : 11]
Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26]
Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#26 ASC NULLS FIRST], false, 0
-(37) Scan parquet default.inventory
-Output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30]
+(38) Scan parquet default.inventory
+Output [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(true)]
+PartitionFilters: [isnotnull(inv_date_sk#31), dynamicpruningexpression(true)]
PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)]
ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int>
-(38) ColumnarToRow [codegen id : 12]
-Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30]
+(39) ColumnarToRow [codegen id : 13]
+Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31]
-(39) Filter [codegen id : 12]
-Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30]
-Condition : ((isnotnull(inv_quantity_on_hand#29) AND isnotnull(inv_item_sk#27)) AND isnotnull(inv_warehouse_sk#28))
+(40) Filter [codegen id : 13]
+Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31]
+Condition : ((isnotnull(inv_quantity_on_hand#30) AND isnotnull(inv_item_sk#28)) AND isnotnull(inv_warehouse_sk#29))
-(40) Scan parquet default.warehouse
-Output [2]: [w_warehouse_sk#31, w_warehouse_name#32]
+(41) Scan parquet default.warehouse
+Output [2]: [w_warehouse_sk#32, w_warehouse_name#33]
Batched: true
Location [not included in comparison]/{warehouse_dir}/warehouse]
PushedFilters: [IsNotNull(w_warehouse_sk)]
ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string>
-(41) ColumnarToRow [codegen id : 11]
-Input [2]: [w_warehouse_sk#31, w_warehouse_name#32]
+(42) ColumnarToRow [codegen id : 12]
+Input [2]: [w_warehouse_sk#32, w_warehouse_name#33]
-(42) Filter [codegen id : 11]
-Input [2]: [w_warehouse_sk#31, w_warehouse_name#32]
-Condition : isnotnull(w_warehouse_sk#31)
+(43) Filter [codegen id : 12]
+Input [2]: [w_warehouse_sk#32, w_warehouse_name#33]
+Condition : isnotnull(w_warehouse_sk#32)
-(43) BroadcastExchange
-Input [2]: [w_warehouse_sk#31, w_warehouse_name#32]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33]
+(44) BroadcastExchange
+Input [2]: [w_warehouse_sk#32, w_warehouse_name#33]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34]
-(44) BroadcastHashJoin [codegen id : 12]
-Left keys [1]: [inv_warehouse_sk#28]
-Right keys [1]: [w_warehouse_sk#31]
+(45) BroadcastHashJoin [codegen id : 13]
+Left keys [1]: [inv_warehouse_sk#29]
+Right keys [1]: [w_warehouse_sk#32]
Join condition: None
-(45) Project [codegen id : 12]
-Output [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
-Input [6]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_sk#31, w_warehouse_name#32]
+(46) Project [codegen id : 13]
+Output [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
+Input [6]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_sk#32, w_warehouse_name#33]
-(46) Exchange
-Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
-Arguments: hashpartitioning(inv_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#34]
+(47) Exchange
+Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
+Arguments: hashpartitioning(inv_item_sk#28, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#35]
-(47) Sort [codegen id : 13]
-Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
-Arguments: [inv_item_sk#27 ASC NULLS FIRST, inv_date_sk#30 ASC NULLS FIRST], false, 0
+(48) Sort [codegen id : 14]
+Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
+Arguments: [inv_item_sk#28 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0
-(48) SortMergeJoin [codegen id : 15]
+(49) SortMergeJoin [codegen id : 16]
Left keys [2]: [cs_item_sk#4, d_date_sk#26]
-Right keys [2]: [inv_item_sk#27, inv_date_sk#30]
-Join condition: (inv_quantity_on_hand#29 < cs_quantity#7)
+Right keys [2]: [inv_item_sk#28, inv_date_sk#31]
+Join condition: (inv_quantity_on_hand#30 < cs_quantity#7)
-(49) Project [codegen id : 15]
-Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
+(50) Project [codegen id : 16]
+Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
-(50) Scan parquet default.promotion
-Output [1]: [p_promo_sk#35]
+(51) Scan parquet default.promotion
+Output [1]: [p_promo_sk#36]
Batched: true
Location [not included in comparison]/{warehouse_dir}/promotion]
PushedFilters: [IsNotNull(p_promo_sk)]
ReadSchema: struct<p_promo_sk:int>
-(51) ColumnarToRow [codegen id : 14]
-Input [1]: [p_promo_sk#35]
+(52) ColumnarToRow [codegen id : 15]
+Input [1]: [p_promo_sk#36]
-(52) Filter [codegen id : 14]
-Input [1]: [p_promo_sk#35]
-Condition : isnotnull(p_promo_sk#35)
+(53) Filter [codegen id : 15]
+Input [1]: [p_promo_sk#36]
+Condition : isnotnull(p_promo_sk#36)
-(53) BroadcastExchange
-Input [1]: [p_promo_sk#35]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36]
+(54) BroadcastExchange
+Input [1]: [p_promo_sk#36]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37]
-(54) BroadcastHashJoin [codegen id : 15]
+(55) BroadcastHashJoin [codegen id : 16]
Left keys [1]: [cs_promo_sk#5]
-Right keys [1]: [p_promo_sk#35]
+Right keys [1]: [p_promo_sk#36]
Join condition: None
-(55) Project [codegen id : 15]
-Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, p_promo_sk#35]
+(56) Project [codegen id : 16]
+Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, p_promo_sk#36]
+
+(57) Exchange
+Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38]
-(56) Sort [codegen id : 15]
-Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
+(58) Sort [codegen id : 17]
+Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0
-(57) Scan parquet default.catalog_returns
-Output [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
+(59) Scan parquet default.catalog_returns
+Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
Batched: true
Location [not included in comparison]/{warehouse_dir}/catalog_returns]
PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)]
ReadSchema: struct<cr_item_sk:int,cr_order_number:int>
-(58) ColumnarToRow [codegen id : 16]
-Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
+(60) ColumnarToRow [codegen id : 18]
+Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
-(59) Filter [codegen id : 16]
-Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
-Condition : (isnotnull(cr_item_sk#37) AND isnotnull(cr_order_number#38))
+(61) Filter [codegen id : 18]
+Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
+Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40))
-(60) Project [codegen id : 16]
-Output [2]: [cr_item_sk#37, cr_order_number#38]
-Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
+(62) Project [codegen id : 18]
+Output [2]: [cr_item_sk#39, cr_order_number#40]
+Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
-(61) Exchange
-Input [2]: [cr_item_sk#37, cr_order_number#38]
-Arguments: hashpartitioning(cr_item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#40]
+(63) Exchange
+Input [2]: [cr_item_sk#39, cr_order_number#40]
+Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42]
-(62) Sort [codegen id : 17]
-Input [2]: [cr_item_sk#37, cr_order_number#38]
-Arguments: [cr_item_sk#37 ASC NULLS FIRST, cr_order_number#38 ASC NULLS FIRST], false, 0
+(64) Sort [codegen id : 19]
+Input [2]: [cr_item_sk#39, cr_order_number#40]
+Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0
-(63) SortMergeJoin [codegen id : 18]
+(65) SortMergeJoin [codegen id : 20]
Left keys [2]: [cs_item_sk#4, cs_order_number#6]
-Right keys [2]: [cr_item_sk#37, cr_order_number#38]
+Right keys [2]: [cr_item_sk#39, cr_order_number#40]
Join condition: None
-(64) Project [codegen id : 18]
-Output [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, cr_item_sk#37, cr_order_number#38]
+(66) Project [codegen id : 20]
+Output [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, cr_item_sk#39, cr_order_number#40]
-(65) HashAggregate [codegen id : 18]
-Input [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25]
+(67) HashAggregate [codegen id : 20]
+Input [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25]
Functions [1]: [partial_count(1)]
-Aggregate Attributes [1]: [count#41]
-Results [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42]
+Aggregate Attributes [1]: [count#43]
+Results [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44]
-(66) Exchange
-Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42]
-Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#32, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#43]
+(68) Exchange
+Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44]
+Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#33, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#45]
-(67) HashAggregate [codegen id : 19]
-Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42]
-Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25]
+(69) HashAggregate [codegen id : 21]
+Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44]
+Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25]
Functions [1]: [count(1)]
-Aggregate Attributes [1]: [count(1)#44]
-Results [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count(1)#44 AS no_promo#45, count(1)#44 AS promo#46, count(1)#44 AS total_cnt#47]
+Aggregate Attributes [1]: [count(1)#46]
+Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49]
-(68) TakeOrderedAndProject
-Input [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47]
-Arguments: 100, [total_cnt#47 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#32 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47]
+(70) TakeOrderedAndProject
+Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
+Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9
-BroadcastExchange (79)
-+- * Project (78)
- +- * BroadcastHashJoin Inner BuildLeft (77)
- :- BroadcastExchange (73)
- : +- * Project (72)
- : +- * Filter (71)
- : +- * ColumnarToRow (70)
- : +- Scan parquet default.date_dim (69)
- +- * Filter (76)
- +- * ColumnarToRow (75)
- +- Scan parquet default.date_dim (74)
-
-
-(69) Scan parquet default.date_dim
-Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
+BroadcastExchange (81)
++- * Project (80)
+ +- * BroadcastHashJoin Inner BuildLeft (79)
+ :- BroadcastExchange (75)
+ : +- * Project (74)
+ : +- * Filter (73)
+ : +- * ColumnarToRow (72)
+ : +- Scan parquet default.date_dim (71)
+ +- * Filter (78)
+ +- * ColumnarToRow (77)
+ +- Scan parquet default.date_dim (76)
+
+
+(71) Scan parquet default.date_dim
+Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)]
ReadSchema: struct<d_date_sk:int,d_date:date,d_week_seq:int,d_year:int>
-(70) ColumnarToRow [codegen id : 1]
-Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
+(72) ColumnarToRow [codegen id : 1]
+Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
-(71) Filter [codegen id : 1]
-Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
-Condition : ((((isnotnull(d_year#48) AND (d_year#48 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24))
+(73) Filter [codegen id : 1]
+Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
+Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24))
-(72) Project [codegen id : 1]
+(74) Project [codegen id : 1]
Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25]
-Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
+Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
-(73) BroadcastExchange
+(75) BroadcastExchange
Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25]
-Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49]
+Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51]
-(74) Scan parquet default.date_dim
-Output [2]: [d_date_sk#26, d_week_seq#50]
+(76) Scan parquet default.date_dim
+Output [2]: [d_date_sk#26, d_week_seq#52]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_week_seq:int>
-(75) ColumnarToRow
-Input [2]: [d_date_sk#26, d_week_seq#50]
+(77) ColumnarToRow
+Input [2]: [d_date_sk#26, d_week_seq#52]
-(76) Filter
-Input [2]: [d_date_sk#26, d_week_seq#50]
-Condition : (isnotnull(d_week_seq#50) AND isnotnull(d_date_sk#26))
+(78) Filter
+Input [2]: [d_date_sk#26, d_week_seq#52]
+Condition : (isnotnull(d_week_seq#52) AND isnotnull(d_date_sk#26))
-(77) BroadcastHashJoin [codegen id : 2]
+(79) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [d_week_seq#25]
-Right keys [1]: [d_week_seq#50]
+Right keys [1]: [d_week_seq#52]
Join condition: None
-(78) Project [codegen id : 2]
+(80) Project [codegen id : 2]
Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
-Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50]
+Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#52]
-(79) BroadcastExchange
+(81) BroadcastExchange
Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt
index d84393b..e838025 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt
@@ -1,126 +1,132 @@
TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo]
- WholeStageCodegen (19)
+ WholeStageCodegen (21)
HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count]
InputAdapter
Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1
- WholeStageCodegen (18)
+ WholeStageCodegen (20)
HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count]
Project [w_warehouse_name,i_item_desc,d_week_seq]
SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number]
InputAdapter
- WholeStageCodegen (15)
+ WholeStageCodegen (17)
Sort [cs_item_sk,cs_order_number]
- Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
- BroadcastHashJoin [cs_promo_sk,p_promo_sk]
- Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
- SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity]
- InputAdapter
- WholeStageCodegen (10)
- Sort [cs_item_sk,d_date_sk]
- Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk]
- BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date]
- Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc]
- SortMergeJoin [cs_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (5)
- Sort [cs_item_sk]
- InputAdapter
- Exchange [cs_item_sk] #2
- WholeStageCodegen (4)
- Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date]
- BroadcastHashJoin [cs_ship_date_sk,d_date_sk]
- Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
- BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
- Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
- BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
- Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #3
- WholeStageCodegen (2)
- Project [d_date_sk,d_date,d_week_seq,d_date_sk]
- BroadcastHashJoin [d_week_seq,d_week_seq]
- InputAdapter
- BroadcastExchange #4
- WholeStageCodegen (1)
- Project [d_date_sk,d_date,d_week_seq]
- Filter [d_year,d_date_sk,d_week_seq,d_date]
- ColumnarToRow
+ InputAdapter
+ Exchange [cs_item_sk,cs_order_number] #2
+ WholeStageCodegen (16)
+ Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
+ BroadcastHashJoin [cs_promo_sk,p_promo_sk]
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
+ SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity]
+ InputAdapter
+ WholeStageCodegen (11)
+ Sort [cs_item_sk,d_date_sk]
+ InputAdapter
+ Exchange [cs_item_sk,d_date_sk] #3
+ WholeStageCodegen (10)
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date]
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc]
+ SortMergeJoin [cs_item_sk,i_item_sk]
+ InputAdapter
+ WholeStageCodegen (5)
+ Sort [cs_item_sk]
+ InputAdapter
+ Exchange [cs_item_sk] #4
+ WholeStageCodegen (4)
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date]
+ BroadcastHashJoin [cs_ship_date_sk,d_date_sk]
+ Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
+ BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
+ BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (2)
+ Project [d_date_sk,d_date,d_week_seq,d_date_sk]
+ BroadcastHashJoin [d_week_seq,d_week_seq]
InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- Filter [d_week_seq,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_week_seq]
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (1)
- Project [hd_demo_sk]
- Filter [hd_buy_potential,hd_demo_sk]
- ColumnarToRow
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ Project [d_date_sk,d_date,d_week_seq]
+ Filter [d_year,d_date_sk,d_week_seq,d_date]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
+ Filter [d_week_seq,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_week_seq]
InputAdapter
- Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential]
- InputAdapter
- BroadcastExchange #6
- WholeStageCodegen (2)
- Project [cd_demo_sk]
- Filter [cd_marital_status,cd_demo_sk]
- ColumnarToRow
+ BroadcastExchange #7
+ WholeStageCodegen (1)
+ Project [hd_demo_sk]
+ Filter [hd_buy_potential,hd_demo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential]
InputAdapter
- Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status]
- InputAdapter
- BroadcastExchange #7
- WholeStageCodegen (3)
- Filter [d_date,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_date]
- InputAdapter
- WholeStageCodegen (7)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #8
- WholeStageCodegen (6)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_item_desc]
- InputAdapter
- ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #3
- InputAdapter
- WholeStageCodegen (13)
- Sort [inv_item_sk,inv_date_sk]
+ BroadcastExchange #8
+ WholeStageCodegen (2)
+ Project [cd_demo_sk]
+ Filter [cd_marital_status,cd_demo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status]
+ InputAdapter
+ BroadcastExchange #9
+ WholeStageCodegen (3)
+ Filter [d_date,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_date]
+ InputAdapter
+ WholeStageCodegen (7)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #10
+ WholeStageCodegen (6)
+ Filter [i_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_item_desc]
+ InputAdapter
+ ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5
InputAdapter
- Exchange [inv_item_sk] #9
- WholeStageCodegen (12)
- Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name]
- BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
- Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- InputAdapter
- BroadcastExchange #10
- WholeStageCodegen (11)
- Filter [w_warehouse_sk]
+ WholeStageCodegen (14)
+ Sort [inv_item_sk,inv_date_sk]
+ InputAdapter
+ Exchange [inv_item_sk,inv_date_sk] #11
+ WholeStageCodegen (13)
+ Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name]
+ BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
ColumnarToRow
InputAdapter
- Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name]
- InputAdapter
- BroadcastExchange #11
- WholeStageCodegen (14)
- Filter [p_promo_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.promotion [p_promo_sk]
+ Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ InputAdapter
+ BroadcastExchange #12
+ WholeStageCodegen (12)
+ Filter [w_warehouse_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name]
+ InputAdapter
+ BroadcastExchange #13
+ WholeStageCodegen (15)
+ Filter [p_promo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.promotion [p_promo_sk]
InputAdapter
- WholeStageCodegen (17)
+ WholeStageCodegen (19)
Sort [cr_item_sk,cr_order_number]
InputAdapter
- Exchange [cr_item_sk] #12
- WholeStageCodegen (16)
+ Exchange [cr_item_sk,cr_order_number] #14
+ WholeStageCodegen (18)
Project [cr_item_sk,cr_order_number]
Filter [cr_item_sk,cr_order_number]
ColumnarToRow
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt
index 332a0b9..c08379b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt
@@ -273,37 +273,34 @@ Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_s
===== Subqueries =====
Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49]
-* HashAggregate (79)
-+- Exchange (78)
- +- * HashAggregate (77)
- +- * HashAggregate (76)
- +- Exchange (75)
- +- * HashAggregate (74)
- +- * Project (73)
- +- * SortMergeJoin Inner (72)
- :- * Sort (65)
- : +- * Project (64)
- : +- * SortMergeJoin Inner (63)
- : :- * Sort (57)
- : : +- Exchange (56)
- : : +- * Project (55)
- : : +- * BroadcastHashJoin Inner BuildLeft (54)
- : : :- ReusedExchange (49)
- : : +- * Project (53)
- : : +- * Filter (52)
- : : +- * ColumnarToRow (51)
- : : +- Scan parquet default.store_sales (50)
- : +- * Sort (62)
- : +- Exchange (61)
- : +- * Filter (60)
- : +- * ColumnarToRow (59)
- : +- Scan parquet default.item (58)
- +- * Sort (71)
- +- Exchange (70)
- +- * Project (69)
- +- * Filter (68)
- +- * ColumnarToRow (67)
- +- Scan parquet default.store_returns (66)
+* HashAggregate (76)
++- Exchange (75)
+ +- * HashAggregate (74)
+ +- * HashAggregate (73)
+ +- Exchange (72)
+ +- * HashAggregate (71)
+ +- * Project (70)
+ +- * SortMergeJoin Inner (69)
+ :- * Sort (66)
+ : +- Exchange (65)
+ : +- * Project (64)
+ : +- * SortMergeJoin Inner (63)
+ : :- * Sort (57)
+ : : +- Exchange (56)
+ : : +- * Project (55)
+ : : +- * BroadcastHashJoin Inner BuildLeft (54)
+ : : :- ReusedExchange (49)
+ : : +- * Project (53)
+ : : +- * Filter (52)
+ : : +- * ColumnarToRow (51)
+ : : +- Scan parquet default.store_sales (50)
+ : +- * Sort (62)
+ : +- Exchange (61)
+ : +- * Filter (60)
+ : +- * ColumnarToRow (59)
+ : +- Scan parquet default.item (58)
+ +- * Sort (68)
+ +- ReusedExchange (67)
(49) ReusedExchange [Reuses operator id: 17]
@@ -375,75 +372,60 @@ Join condition: None
Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29]
Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29]
-(65) Sort [codegen id : 8]
+(65) Exchange
Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29]
-Arguments: [ss_ticket_number#21 ASC NULLS FIRST, ss_item_sk#18 ASC NULLS FIRST], false, 0
-
-(66) Scan parquet default.store_returns
-Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34]
-Batched: true
-Location [not included in comparison]/{warehouse_dir}/store_returns]
-PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)]
-ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int>
-
-(67) ColumnarToRow [codegen id : 9]
-Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34]
+Arguments: hashpartitioning(ss_ticket_number#21, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#53]
-(68) Filter [codegen id : 9]
-Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34]
-Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32))
+(66) Sort [codegen id : 9]
+Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29]
+Arguments: [ss_ticket_number#21 ASC NULLS FIRST, ss_item_sk#18 ASC NULLS FIRST], false, 0
-(69) Project [codegen id : 9]
+(67) ReusedExchange [Reuses operator id: 36]
Output [2]: [sr_item_sk#32, sr_ticket_number#33]
-Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34]
-(70) Exchange
-Input [2]: [sr_item_sk#32, sr_ticket_number#33]
-Arguments: hashpartitioning(sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#53]
-
-(71) Sort [codegen id : 10]
+(68) Sort [codegen id : 11]
Input [2]: [sr_item_sk#32, sr_ticket_number#33]
Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0
-(72) SortMergeJoin [codegen id : 11]
+(69) SortMergeJoin [codegen id : 12]
Left keys [2]: [ss_ticket_number#21, ss_item_sk#18]
Right keys [2]: [sr_ticket_number#33, sr_item_sk#32]
Join condition: None
-(73) Project [codegen id : 11]
+(70) Project [codegen id : 12]
Output [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8]
Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, sr_item_sk#32, sr_ticket_number#33]
-(74) HashAggregate [codegen id : 11]
+(71) HashAggregate [codegen id : 12]
Input [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8]
Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26]
Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))]
Aggregate Attributes [1]: [sum#54]
Results [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55]
-(75) Exchange
+(72) Exchange
Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55]
Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, 5), ENSURE_REQUIREMENTS, [id=#56]
-(76) HashAggregate [codegen id : 12]
+(73) HashAggregate [codegen id : 13]
Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55]
Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26]
Functions [1]: [sum(UnscaledValue(ss_net_paid#22))]
Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#39]
Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#39,17,2) AS netpaid#40]
-(77) HashAggregate [codegen id : 12]
+(74) HashAggregate [codegen id : 13]
Input [1]: [netpaid#40]
Keys: []
Functions [1]: [partial_avg(netpaid#40)]
Aggregate Attributes [2]: [sum#57, count#58]
Results [2]: [sum#59, count#60]
-(78) Exchange
+(75) Exchange
Input [2]: [sum#59, count#60]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61]
-(79) HashAggregate [codegen id : 13]
+(76) HashAggregate [codegen id : 14]
Input [2]: [sum#59, count#60]
Keys: []
Functions [1]: [avg(netpaid#40)]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt
index d12b734..4beebcb 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt
@@ -5,60 +5,57 @@ WholeStageCodegen (12)
WholeStageCodegen (11)
Filter [paid]
Subquery #1
- WholeStageCodegen (13)
+ WholeStageCodegen (14)
HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count]
InputAdapter
Exchange #10
- WholeStageCodegen (12)
+ WholeStageCodegen (13)
HashAggregate [netpaid] [sum,count,sum,count]
HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum]
InputAdapter
Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11
- WholeStageCodegen (11)
+ WholeStageCodegen (12)
HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum]
Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state]
SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
InputAdapter
- WholeStageCodegen (8)
+ WholeStageCodegen (9)
Sort [ss_ticket_number,ss_item_sk]
- Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id]
- SortMergeJoin [ss_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (5)
- Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_ticket_number,ss_item_sk] #12
+ WholeStageCodegen (8)
+ Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id]
+ SortMergeJoin [ss_item_sk,i_item_sk]
InputAdapter
- Exchange [ss_item_sk] #12
- WholeStageCodegen (4)
- Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid]
- BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk]
- InputAdapter
- ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5
- Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid]
- Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk]
- ColumnarToRow
+ WholeStageCodegen (5)
+ Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_item_sk] #13
+ WholeStageCodegen (4)
+ Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid]
+ BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk]
InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk]
- InputAdapter
- WholeStageCodegen (7)
- Sort [i_item_sk]
+ ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid]
+ Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk]
InputAdapter
- Exchange [i_item_sk] #13
- WholeStageCodegen (6)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
+ WholeStageCodegen (7)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #14
+ WholeStageCodegen (6)
+ Filter [i_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
InputAdapter
- WholeStageCodegen (10)
+ WholeStageCodegen (11)
Sort [sr_ticket_number,sr_item_sk]
InputAdapter
- Exchange [sr_item_sk] #14
- WholeStageCodegen (9)
- Project [sr_item_sk,sr_ticket_number]
- Filter [sr_ticket_number,sr_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
+ ReusedExchange [sr_item_sk,sr_ticket_number] #9
HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty]
InputAdapter
Exchange [c_last_name,c_first_name,s_store_name] #2
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
index 51b2f05..4566f30 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
@@ -1,53 +1,56 @@
== Physical Plan ==
-TakeOrderedAndProject (49)
-+- * Project (48)
- +- * SortMergeJoin Inner (47)
- :- * Project (41)
- : +- * SortMergeJoin Inner (40)
- : :- * Sort (32)
- : : +- * Project (31)
- : : +- * Filter (30)
- : : +- Window (29)
- : : +- * Filter (28)
- : : +- Window (27)
- : : +- * Sort (26)
- : : +- Exchange (25)
- : : +- * HashAggregate (24)
- : : +- Exchange (23)
- : : +- * HashAggregate (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.store_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.store (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (39)
- : +- * Project (38)
- : +- Window (37)
- : +- * Sort (36)
- : +- Exchange (35)
- : +- * HashAggregate (34)
- : +- ReusedExchange (33)
- +- * Sort (46)
- +- * Project (45)
- +- Window (44)
- +- * Sort (43)
- +- ReusedExchange (42)
+TakeOrderedAndProject (52)
++- * Project (51)
+ +- * SortMergeJoin Inner (50)
+ :- * Project (43)
+ : +- * SortMergeJoin Inner (42)
+ : :- * Sort (33)
+ : : +- Exchange (32)
+ : : +- * Project (31)
+ : : +- * Filter (30)
+ : : +- Window (29)
+ : : +- * Filter (28)
+ : : +- Window (27)
+ : : +- * Sort (26)
+ : : +- Exchange (25)
+ : : +- * HashAggregate (24)
+ : : +- Exchange (23)
+ : : +- * HashAggregate (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.store_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.store (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (41)
+ : +- Exchange (40)
+ : +- * Project (39)
+ : +- Window (38)
+ : +- * Sort (37)
+ : +- Exchange (36)
+ : +- * HashAggregate (35)
+ : +- ReusedExchange (34)
+ +- * Sort (49)
+ +- Exchange (48)
+ +- * Project (47)
+ +- Window (46)
+ +- * Sort (45)
+ +- ReusedExchange (44)
(1) Scan parquet default.store_sales
@@ -65,7 +68,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2))
-(4) ReusedExchange [Reuses operator id: 53]
+(4) ReusedExchange [Reuses operator id: 56]
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -189,106 +192,118 @@ Condition : ((isnotnull(avg_monthly_sales#26) AND (avg_monthly_sales#26 > 0.0000
Output [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25]
Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, _w0#23, rn#25, avg_monthly_sales#26]
-(32) Sort [codegen id : 11]
+(32) Exchange
+Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25]
+Arguments: hashpartitioning(i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25, 5), ENSURE_REQUIREMENTS, [id=#27]
+
+(33) Sort [codegen id : 12]
Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25]
Arguments: [i_category#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST, s_company_name#11 ASC NULLS FIRST, rn#25 ASC NULLS FIRST], false, 0
-(33) ReusedExchange [Reuses operator id: 23]
-Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33]
+(34) ReusedExchange [Reuses operator id: 23]
+Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34]
-(34) HashAggregate [codegen id : 19]
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33]
-Keys [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32]
-Functions [1]: [sum(UnscaledValue(ss_sales_price#34))]
-Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#34))#21]
-Results [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#34))#21,17,2) AS sum_sales#22]
+(35) HashAggregate [codegen id : 20]
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34]
+Keys [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33]
+Functions [1]: [sum(UnscaledValue(ss_sales_price#35))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#21]
+Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#21,17,2) AS sum_sales#22]
-(35) Exchange
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22]
-Arguments: hashpartitioning(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, 5), ENSURE_REQUIREMENTS, [id=#35]
+(36) Exchange
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22]
+Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#36]
-(36) Sort [codegen id : 20]
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22]
-Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST], false, 0
+(37) Sort [codegen id : 21]
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22]
+Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0
-(37) Window
-Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22]
-Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#27, i_brand#28, s_store_name#29, s_company_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST]
+(38) Window
+Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22]
+Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST]
-(38) Project [codegen id : 21]
-Output [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#22 AS sum_sales#37, rn#36]
-Input [8]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22, rn#36]
+(39) Project [codegen id : 22]
+Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#22 AS sum_sales#38, rn#37]
+Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22, rn#37]
-(39) Sort [codegen id : 21]
-Input [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36]
-Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, (rn#36 + 1) ASC NULLS FIRST], false, 0
+(40) Exchange
+Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37]
+Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#39]
-(40) SortMergeJoin [codegen id : 22]
+(41) Sort [codegen id : 23]
+Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37]
+Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0
+
+(42) SortMergeJoin [codegen id : 24]
Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25]
-Right keys [5]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, (rn#36 + 1)]
+Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1)]
Join condition: None
-(41) Project [codegen id : 22]
-Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37]
-Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36]
+(43) Project [codegen id : 24]
+Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38]
+Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37]
+
+(44) ReusedExchange [Reuses operator id: 36]
+Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22]
-(42) ReusedExchange [Reuses operator id: 35]
-Output [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22]
+(45) Sort [codegen id : 33]
+Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22]
+Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0
-(43) Sort [codegen id : 31]
-Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22]
-Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0
+(46) Window
+Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22]
+Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST]
-(44) Window
-Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22]
-Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, s_store_name#40, s_company_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST]
+(47) Project [codegen id : 34]
+Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#22 AS sum_sales#47, rn#46]
+Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22, rn#46]
-(45) Project [codegen id : 32]
-Output [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#22 AS sum_sales#45, rn#44]
-Input [8]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22, rn#44]
+(48) Exchange
+Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46]
+Arguments: hashpartitioning(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#48]
-(46) Sort [codegen id : 32]
-Input [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44]
-Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, (rn#44 - 1) ASC NULLS FIRST], false, 0
+(49) Sort [codegen id : 35]
+Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46]
+Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0
-(47) SortMergeJoin [codegen id : 33]
+(50) SortMergeJoin [codegen id : 36]
Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25]
-Right keys [5]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, (rn#44 - 1)]
+Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1)]
Join condition: None
-(48) Project [codegen id : 33]
-Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#46, sum_sales#45 AS nsum#47]
-Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44]
+(51) Project [codegen id : 36]
+Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#38 AS psum#49, sum_sales#47 AS nsum#50]
+Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46]
-(49) TakeOrderedAndProject
-Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47]
+(52) TakeOrderedAndProject
+Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
+Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (53)
-+- * Filter (52)
- +- * ColumnarToRow (51)
- +- Scan parquet default.date_dim (50)
+BroadcastExchange (56)
++- * Filter (55)
+ +- * ColumnarToRow (54)
+ +- Scan parquet default.date_dim (53)
-(50) Scan parquet default.date_dim
+(53) Scan parquet default.date_dim
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(51) ColumnarToRow [codegen id : 1]
+(54) ColumnarToRow [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-(52) Filter [codegen id : 1]
+(55) Filter [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6))
-(53) BroadcastExchange
+(56) BroadcastExchange
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt
index 65bcf10..5f64a22 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt
@@ -1,95 +1,104 @@
TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum]
- WholeStageCodegen (33)
+ WholeStageCodegen (36)
Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales]
SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn]
InputAdapter
- WholeStageCodegen (22)
+ WholeStageCodegen (24)
Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales]
SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn]
InputAdapter
- WholeStageCodegen (11)
+ WholeStageCodegen (12)
Sort [i_category,i_brand,s_store_name,s_company_name,rn]
- Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
- Filter [avg_monthly_sales,sum_sales]
- InputAdapter
- Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year]
- WholeStageCodegen (10)
- Filter [d_year]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
- WholeStageCodegen (9)
- Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,s_store_name,s_company_name] #1
- WholeStageCodegen (8)
- HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum]
- InputAdapter
- Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2
- WholeStageCodegen (7)
- HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum]
- Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- SortMergeJoin [ss_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [ss_item_sk]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #1
+ WholeStageCodegen (11)
+ Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
+ Filter [avg_monthly_sales,sum_sales]
+ InputAdapter
+ Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year]
+ WholeStageCodegen (10)
+ Filter [d_year]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
+ WholeStageCodegen (9)
+ Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name] #2
+ WholeStageCodegen (8)
+ HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3
+ WholeStageCodegen (7)
+ HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum]
+ Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
+ SortMergeJoin [ss_item_sk,i_item_sk]
InputAdapter
- Exchange [ss_item_sk] #3
- WholeStageCodegen (3)
- Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Filter [ss_item_sk,ss_store_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #4
- WholeStageCodegen (1)
- Filter [d_year,d_moy,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
- InputAdapter
- ReusedExchange [d_date_sk,d_year,d_moy] #4
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (2)
- Filter [s_store_sk,s_store_name,s_company_name]
- ColumnarToRow
+ WholeStageCodegen (4)
+ Sort [ss_item_sk]
+ InputAdapter
+ Exchange [ss_item_sk] #4
+ WholeStageCodegen (3)
+ Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Filter [ss_item_sk,ss_store_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ Filter [d_year,d_moy,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
- Scan parquet default.store [s_store_sk,s_store_name,s_company_name]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
+ ReusedExchange [d_date_sk,d_year,d_moy] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [s_store_sk,s_store_name,s_company_name]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
- Exchange [i_item_sk] #6
- WholeStageCodegen (5)
- Filter [i_item_sk,i_category,i_brand]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_brand,i_category]
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk,i_category,i_brand]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_brand,i_category]
InputAdapter
- WholeStageCodegen (21)
+ WholeStageCodegen (23)
Sort [i_category,i_brand,s_store_name,s_company_name,rn]
- Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
- WholeStageCodegen (20)
- Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,s_store_name,s_company_name] #7
- WholeStageCodegen (19)
- HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum]
- InputAdapter
- ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8
+ WholeStageCodegen (22)
+ Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
+ WholeStageCodegen (21)
+ Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name] #9
+ WholeStageCodegen (20)
+ HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3
InputAdapter
- WholeStageCodegen (32)
+ WholeStageCodegen (35)
Sort [i_category,i_brand,s_store_name,s_company_name,rn]
- Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
- WholeStageCodegen (31)
- Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
- InputAdapter
- ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7
+ InputAdapter
+ Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10
+ WholeStageCodegen (34)
+ Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name]
+ WholeStageCodegen (33)
+ Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #9
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt
index e3d76bf..64111ee 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt
@@ -1,72 +1,74 @@
== Physical Plan ==
-TakeOrderedAndProject (68)
-+- * Filter (67)
- +- * HashAggregate (66)
- +- * HashAggregate (65)
- +- * Project (64)
- +- * SortMergeJoin Inner (63)
- :- Window (58)
- : +- * Sort (57)
- : +- Exchange (56)
- : +- * Project (55)
- : +- * Filter (54)
- : +- * SortMergeJoin FullOuter (53)
- : :- * Sort (26)
- : : +- * HashAggregate (25)
- : : +- * HashAggregate (24)
- : : +- * Project (23)
- : : +- * SortMergeJoin Inner (22)
- : : :- * Sort (15)
- : : : +- Exchange (14)
- : : : +- * Project (13)
- : : : +- Window (12)
- : : : +- * Sort (11)
- : : : +- Exchange (10)
- : : : +- * HashAggregate (9)
- : : : +- Exchange (8)
- : : : +- * HashAggregate (7)
- : : : +- * Project (6)
- : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : :- * Filter (3)
- : : : : +- * ColumnarToRow (2)
- : : : : +- Scan parquet default.web_sales (1)
- : : : +- ReusedExchange (4)
- : : +- * Sort (21)
- : : +- Exchange (20)
- : : +- * Project (19)
- : : +- Window (18)
- : : +- * Sort (17)
- : : +- ReusedExchange (16)
- : +- * Sort (52)
- : +- * HashAggregate (51)
- : +- * HashAggregate (50)
- : +- * Project (49)
- : +- * SortMergeJoin Inner (48)
- : :- * Sort (41)
- : : +- Exchange (40)
- : : +- * Project (39)
- : : +- Window (38)
- : : +- * Sort (37)
- : : +- Exchange (36)
- : : +- * HashAggregate (35)
- : : +- Exchange (34)
- : : +- * HashAggregate (33)
- : : +- * Project (32)
- : : +- * BroadcastHashJoin Inner BuildRight (31)
- : : :- * Filter (29)
- : : : +- * ColumnarToRow (28)
- : : : +- Scan parquet default.store_sales (27)
- : : +- ReusedExchange (30)
- : +- * Sort (47)
- : +- Exchange (46)
- : +- * Project (45)
- : +- Window (44)
- : +- * Sort (43)
- : +- ReusedExchange (42)
- +- * Project (62)
- +- Window (61)
- +- * Sort (60)
- +- ReusedExchange (59)
+TakeOrderedAndProject (70)
++- * Filter (69)
+ +- * HashAggregate (68)
+ +- * HashAggregate (67)
+ +- * Project (66)
+ +- * SortMergeJoin Inner (65)
+ :- Window (60)
+ : +- * Sort (59)
+ : +- Exchange (58)
+ : +- * Project (57)
+ : +- * Filter (56)
+ : +- * SortMergeJoin FullOuter (55)
+ : :- * Sort (27)
+ : : +- Exchange (26)
+ : : +- * HashAggregate (25)
+ : : +- * HashAggregate (24)
+ : : +- * Project (23)
+ : : +- * SortMergeJoin Inner (22)
+ : : :- * Sort (15)
+ : : : +- Exchange (14)
+ : : : +- * Project (13)
+ : : : +- Window (12)
+ : : : +- * Sort (11)
+ : : : +- Exchange (10)
+ : : : +- * HashAggregate (9)
+ : : : +- Exchange (8)
+ : : : +- * HashAggregate (7)
+ : : : +- * Project (6)
+ : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : :- * Filter (3)
+ : : : : +- * ColumnarToRow (2)
+ : : : : +- Scan parquet default.web_sales (1)
+ : : : +- ReusedExchange (4)
+ : : +- * Sort (21)
+ : : +- Exchange (20)
+ : : +- * Project (19)
+ : : +- Window (18)
+ : : +- * Sort (17)
+ : : +- ReusedExchange (16)
+ : +- * Sort (54)
+ : +- Exchange (53)
+ : +- * HashAggregate (52)
+ : +- * HashAggregate (51)
+ : +- * Project (50)
+ : +- * SortMergeJoin Inner (49)
+ : :- * Sort (42)
+ : : +- Exchange (41)
+ : : +- * Project (40)
+ : : +- Window (39)
+ : : +- * Sort (38)
+ : : +- Exchange (37)
+ : : +- * HashAggregate (36)
+ : : +- Exchange (35)
+ : : +- * HashAggregate (34)
+ : : +- * Project (33)
+ : : +- * BroadcastHashJoin Inner BuildRight (32)
+ : : :- * Filter (30)
+ : : : +- * ColumnarToRow (29)
+ : : : +- Scan parquet default.store_sales (28)
+ : : +- ReusedExchange (31)
+ : +- * Sort (48)
+ : +- Exchange (47)
+ : +- * Project (46)
+ : +- Window (45)
+ : +- * Sort (44)
+ : +- ReusedExchange (43)
+ +- * Project (64)
+ +- Window (63)
+ +- * Sort (62)
+ +- ReusedExchange (61)
(1) Scan parquet default.web_sales
@@ -84,7 +86,7 @@ Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3]
Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3]
Condition : isnotnull(ws_item_sk#1)
-(4) ReusedExchange [Reuses operator id: 73]
+(4) ReusedExchange [Reuses operator id: 75]
Output [2]: [d_date_sk#5, d_date#6]
(5) BroadcastHashJoin [codegen id : 2]
@@ -184,232 +186,240 @@ Functions [1]: [sum(sumws#20)]
Aggregate Attributes [1]: [sum(sumws#20)#26]
Results [3]: [item_sk#11, d_date#6, sum(sumws#20)#26 AS cume_sales#27]
-(26) Sort [codegen id : 13]
+(26) Exchange
+Input [3]: [item_sk#11, d_date#6, cume_sales#27]
+Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#28]
+
+(27) Sort [codegen id : 14]
Input [3]: [item_sk#11, d_date#6, cume_sales#27]
Arguments: [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0
-(27) Scan parquet default.store_sales
-Output [3]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30]
+(28) Scan parquet default.store_sales
+Output [3]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#4)]
+PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#4)]
PushedFilters: [IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_sales_price:decimal(7,2)>
-(28) ColumnarToRow [codegen id : 15]
-Input [3]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30]
+(29) ColumnarToRow [codegen id : 16]
+Input [3]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31]
-(29) Filter [codegen id : 15]
-Input [3]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30]
-Condition : isnotnull(ss_item_sk#28)
+(30) Filter [codegen id : 16]
+Input [3]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31]
+Condition : isnotnull(ss_item_sk#29)
-(30) ReusedExchange [Reuses operator id: 73]
-Output [2]: [d_date_sk#31, d_date#32]
+(31) ReusedExchange [Reuses operator id: 75]
+Output [2]: [d_date_sk#32, d_date#33]
-(31) BroadcastHashJoin [codegen id : 15]
-Left keys [1]: [ss_sold_date_sk#30]
-Right keys [1]: [d_date_sk#31]
+(32) BroadcastHashJoin [codegen id : 16]
+Left keys [1]: [ss_sold_date_sk#31]
+Right keys [1]: [d_date_sk#32]
Join condition: None
-(32) Project [codegen id : 15]
-Output [3]: [ss_item_sk#28, ss_sales_price#29, d_date#32]
-Input [5]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30, d_date_sk#31, d_date#32]
-
-(33) HashAggregate [codegen id : 15]
-Input [3]: [ss_item_sk#28, ss_sales_price#29, d_date#32]
-Keys [2]: [ss_item_sk#28, d_date#32]
-Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#29))]
-Aggregate Attributes [1]: [sum#33]
-Results [3]: [ss_item_sk#28, d_date#32, sum#34]
-
-(34) Exchange
-Input [3]: [ss_item_sk#28, d_date#32, sum#34]
-Arguments: hashpartitioning(ss_item_sk#28, d_date#32, 5), ENSURE_REQUIREMENTS, [id=#35]
-
-(35) HashAggregate [codegen id : 16]
-Input [3]: [ss_item_sk#28, d_date#32, sum#34]
-Keys [2]: [ss_item_sk#28, d_date#32]
-Functions [1]: [sum(UnscaledValue(ss_sales_price#29))]
-Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#36]
-Results [4]: [ss_item_sk#28 AS item_sk#37, d_date#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#36,17,2) AS sumss#38, ss_item_sk#28]
-
-(36) Exchange
-Input [4]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28]
-Arguments: hashpartitioning(ss_item_sk#28, 5), ENSURE_REQUIREMENTS, [id=#39]
-
-(37) Sort [codegen id : 17]
-Input [4]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28]
-Arguments: [ss_item_sk#28 ASC NULLS FIRST, d_date#32 ASC NULLS FIRST], false, 0
-
-(38) Window
-Input [4]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28]
-Arguments: [row_number() windowspecdefinition(ss_item_sk#28, d_date#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#40], [ss_item_sk#28], [d_date#32 ASC NULLS FIRST]
-
-(39) Project [codegen id : 18]
-Output [4]: [item_sk#37, d_date#32, sumss#38, rk#40]
-Input [5]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28, rk#40]
-
-(40) Exchange
-Input [4]: [item_sk#37, d_date#32, sumss#38, rk#40]
-Arguments: hashpartitioning(item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41]
-
-(41) Sort [codegen id : 19]
-Input [4]: [item_sk#37, d_date#32, sumss#38, rk#40]
-Arguments: [item_sk#37 ASC NULLS FIRST], false, 0
-
-(42) ReusedExchange [Reuses operator id: 36]
-Output [4]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43]
-
-(43) Sort [codegen id : 23]
-Input [4]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43]
-Arguments: [ss_item_sk#43 ASC NULLS FIRST, d_date#42 ASC NULLS FIRST], false, 0
-
-(44) Window
-Input [4]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43]
-Arguments: [row_number() windowspecdefinition(ss_item_sk#43, d_date#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [ss_item_sk#43], [d_date#42 ASC NULLS FIRST]
-
-(45) Project [codegen id : 24]
-Output [3]: [item_sk#37 AS item_sk#45, sumss#38 AS sumss#46, rk#44]
-Input [5]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43, rk#44]
-
-(46) Exchange
-Input [3]: [item_sk#45, sumss#46, rk#44]
-Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#47]
-
-(47) Sort [codegen id : 25]
-Input [3]: [item_sk#45, sumss#46, rk#44]
-Arguments: [item_sk#45 ASC NULLS FIRST], false, 0
-
-(48) SortMergeJoin [codegen id : 26]
-Left keys [1]: [item_sk#37]
-Right keys [1]: [item_sk#45]
-Join condition: (rk#40 >= rk#44)
-
-(49) Project [codegen id : 26]
-Output [4]: [item_sk#37, d_date#32, sumss#38, sumss#46]
-Input [7]: [item_sk#37, d_date#32, sumss#38, rk#40, item_sk#45, sumss#46, rk#44]
-
-(50) HashAggregate [codegen id : 26]
-Input [4]: [item_sk#37, d_date#32, sumss#38, sumss#46]
-Keys [3]: [item_sk#37, d_date#32, sumss#38]
-Functions [1]: [partial_sum(sumss#46)]
-Aggregate Attributes [2]: [sum#48, isEmpty#49]
-Results [5]: [item_sk#37, d_date#32, sumss#38, sum#50, isEmpty#51]
-
-(51) HashAggregate [codegen id : 26]
-Input [5]: [item_sk#37, d_date#32, sumss#38, sum#50, isEmpty#51]
-Keys [3]: [item_sk#37, d_date#32, sumss#38]
-Functions [1]: [sum(sumss#46)]
-Aggregate Attributes [1]: [sum(sumss#46)#52]
-Results [3]: [item_sk#37, d_date#32, sum(sumss#46)#52 AS cume_sales#53]
-
-(52) Sort [codegen id : 26]
-Input [3]: [item_sk#37, d_date#32, cume_sales#53]
-Arguments: [item_sk#37 ASC NULLS FIRST, d_date#32 ASC NULLS FIRST], false, 0
-
-(53) SortMergeJoin [codegen id : 27]
+(33) Project [codegen id : 16]
+Output [3]: [ss_item_sk#29, ss_sales_price#30, d_date#33]
+Input [5]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31, d_date_sk#32, d_date#33]
+
+(34) HashAggregate [codegen id : 16]
+Input [3]: [ss_item_sk#29, ss_sales_price#30, d_date#33]
+Keys [2]: [ss_item_sk#29, d_date#33]
+Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#30))]
+Aggregate Attributes [1]: [sum#34]
+Results [3]: [ss_item_sk#29, d_date#33, sum#35]
+
+(35) Exchange
+Input [3]: [ss_item_sk#29, d_date#33, sum#35]
+Arguments: hashpartitioning(ss_item_sk#29, d_date#33, 5), ENSURE_REQUIREMENTS, [id=#36]
+
+(36) HashAggregate [codegen id : 17]
+Input [3]: [ss_item_sk#29, d_date#33, sum#35]
+Keys [2]: [ss_item_sk#29, d_date#33]
+Functions [1]: [sum(UnscaledValue(ss_sales_price#30))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#37]
+Results [4]: [ss_item_sk#29 AS item_sk#38, d_date#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#37,17,2) AS sumss#39, ss_item_sk#29]
+
+(37) Exchange
+Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29]
+Arguments: hashpartitioning(ss_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#40]
+
+(38) Sort [codegen id : 18]
+Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29]
+Arguments: [ss_item_sk#29 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST], false, 0
+
+(39) Window
+Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29]
+Arguments: [row_number() windowspecdefinition(ss_item_sk#29, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#41], [ss_item_sk#29], [d_date#33 ASC NULLS FIRST]
+
+(40) Project [codegen id : 19]
+Output [4]: [item_sk#38, d_date#33, sumss#39, rk#41]
+Input [5]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29, rk#41]
+
+(41) Exchange
+Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41]
+Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, [id=#42]
+
+(42) Sort [codegen id : 20]
+Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41]
+Arguments: [item_sk#38 ASC NULLS FIRST], false, 0
+
+(43) ReusedExchange [Reuses operator id: 37]
+Output [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44]
+
+(44) Sort [codegen id : 24]
+Input [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44]
+Arguments: [ss_item_sk#44 ASC NULLS FIRST, d_date#43 ASC NULLS FIRST], false, 0
+
+(45) Window
+Input [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44]
+Arguments: [row_number() windowspecdefinition(ss_item_sk#44, d_date#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#45], [ss_item_sk#44], [d_date#43 ASC NULLS FIRST]
+
+(46) Project [codegen id : 25]
+Output [3]: [item_sk#38 AS item_sk#46, sumss#39 AS sumss#47, rk#45]
+Input [5]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44, rk#45]
+
+(47) Exchange
+Input [3]: [item_sk#46, sumss#47, rk#45]
+Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#48]
+
+(48) Sort [codegen id : 26]
+Input [3]: [item_sk#46, sumss#47, rk#45]
+Arguments: [item_sk#46 ASC NULLS FIRST], false, 0
+
+(49) SortMergeJoin [codegen id : 27]
+Left keys [1]: [item_sk#38]
+Right keys [1]: [item_sk#46]
+Join condition: (rk#41 >= rk#45)
+
+(50) Project [codegen id : 27]
+Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#47]
+Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#46, sumss#47, rk#45]
+
+(51) HashAggregate [codegen id : 27]
+Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#47]
+Keys [3]: [item_sk#38, d_date#33, sumss#39]
+Functions [1]: [partial_sum(sumss#47)]
+Aggregate Attributes [2]: [sum#49, isEmpty#50]
+Results [5]: [item_sk#38, d_date#33, sumss#39, sum#51, isEmpty#52]
+
+(52) HashAggregate [codegen id : 27]
+Input [5]: [item_sk#38, d_date#33, sumss#39, sum#51, isEmpty#52]
+Keys [3]: [item_sk#38, d_date#33, sumss#39]
+Functions [1]: [sum(sumss#47)]
+Aggregate Attributes [1]: [sum(sumss#47)#53]
+Results [3]: [item_sk#38, d_date#33, sum(sumss#47)#53 AS cume_sales#54]
+
+(53) Exchange
+Input [3]: [item_sk#38, d_date#33, cume_sales#54]
+Arguments: hashpartitioning(item_sk#38, d_date#33, 5), ENSURE_REQUIREMENTS, [id=#55]
+
+(54) Sort [codegen id : 28]
+Input [3]: [item_sk#38, d_date#33, cume_sales#54]
+Arguments: [item_sk#38 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST], false, 0
+
+(55) SortMergeJoin [codegen id : 29]
Left keys [2]: [item_sk#11, d_date#6]
-Right keys [2]: [item_sk#37, d_date#32]
+Right keys [2]: [item_sk#38, d_date#33]
Join condition: None
-(54) Filter [codegen id : 27]
-Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#37, d_date#32, cume_sales#53]
-Condition : isnotnull(CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#37 END)
+(56) Filter [codegen id : 29]
+Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#38, d_date#33, cume_sales#54]
+Condition : isnotnull(CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#38 END)
-(55) Project [codegen id : 27]
-Output [4]: [CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#37 END AS item_sk#54, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#32 END AS d_date#55, cume_sales#27 AS web_sales#56, cume_sales#53 AS store_sales#57]
-Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#37, d_date#32, cume_sales#53]
+(57) Project [codegen id : 29]
+Output [4]: [CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#38 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#33 END AS d_date#57, cume_sales#27 AS web_sales#58, cume_sales#54 AS store_sales#59]
+Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#38, d_date#33, cume_sales#54]
-(56) Exchange
-Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Arguments: hashpartitioning(item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58]
+(58) Exchange
+Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60]
-(57) Sort [codegen id : 28]
-Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Arguments: [item_sk#54 ASC NULLS FIRST, d_date#55 ASC NULLS FIRST], false, 0
+(59) Sort [codegen id : 30]
+Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0
-(58) Window
-Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Arguments: [row_number() windowspecdefinition(item_sk#54, d_date#55 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#59], [item_sk#54], [d_date#55 ASC NULLS FIRST]
+(60) Window
+Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST]
-(59) ReusedExchange [Reuses operator id: 56]
-Output [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
+(61) ReusedExchange [Reuses operator id: 58]
+Output [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
-(60) Sort [codegen id : 56]
-Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Arguments: [item_sk#54 ASC NULLS FIRST, d_date#55 ASC NULLS FIRST], false, 0
+(62) Sort [codegen id : 60]
+Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0
-(61) Window
-Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Arguments: [row_number() windowspecdefinition(item_sk#54, d_date#55 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#60], [item_sk#54], [d_date#55 ASC NULLS FIRST]
+(63) Window
+Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#62], [item_sk#56], [d_date#57 ASC NULLS FIRST]
-(62) Project [codegen id : 57]
-Output [4]: [item_sk#54 AS item_sk#61, web_sales#56 AS web_sales#62, store_sales#57 AS store_sales#63, rk#60]
-Input [5]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, rk#60]
+(64) Project [codegen id : 61]
+Output [4]: [item_sk#56 AS item_sk#63, web_sales#58 AS web_sales#64, store_sales#59 AS store_sales#65, rk#62]
+Input [5]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#62]
-(63) SortMergeJoin [codegen id : 58]
-Left keys [1]: [item_sk#54]
-Right keys [1]: [item_sk#61]
-Join condition: (rk#59 >= rk#60)
+(65) SortMergeJoin [codegen id : 62]
+Left keys [1]: [item_sk#56]
+Right keys [1]: [item_sk#63]
+Join condition: (rk#61 >= rk#62)
-(64) Project [codegen id : 58]
-Output [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_sales#62, store_sales#63]
-Input [9]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, rk#59, item_sk#61, web_sales#62, store_sales#63, rk#60]
+(66) Project [codegen id : 62]
+Output [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65]
+Input [9]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#61, item_sk#63, web_sales#64, store_sales#65, rk#62]
-(65) HashAggregate [codegen id : 58]
-Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_sales#62, store_sales#63]
-Keys [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Functions [2]: [partial_max(web_sales#62), partial_max(store_sales#63)]
-Aggregate Attributes [2]: [max#64, max#65]
-Results [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, max#66, max#67]
+(67) HashAggregate [codegen id : 62]
+Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65]
+Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Functions [2]: [partial_max(web_sales#64), partial_max(store_sales#65)]
+Aggregate Attributes [2]: [max#66, max#67]
+Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#68, max#69]
-(66) HashAggregate [codegen id : 58]
-Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, max#66, max#67]
-Keys [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57]
-Functions [2]: [max(web_sales#62), max(store_sales#63)]
-Aggregate Attributes [2]: [max(web_sales#62)#68, max(store_sales#63)#69]
-Results [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, max(web_sales#62)#68 AS web_cumulative#70, max(store_sales#63)#69 AS store_cumulative#71]
+(68) HashAggregate [codegen id : 62]
+Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#68, max#69]
+Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59]
+Functions [2]: [max(web_sales#64), max(store_sales#65)]
+Aggregate Attributes [2]: [max(web_sales#64)#70, max(store_sales#65)#71]
+Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max(web_sales#64)#70 AS web_cumulative#72, max(store_sales#65)#71 AS store_cumulative#73]
-(67) Filter [codegen id : 58]
-Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_cumulative#70, store_cumulative#71]
-Condition : ((isnotnull(web_cumulative#70) AND isnotnull(store_cumulative#71)) AND (web_cumulative#70 > store_cumulative#71))
+(69) Filter [codegen id : 62]
+Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73]
+Condition : ((isnotnull(web_cumulative#72) AND isnotnull(store_cumulative#73)) AND (web_cumulative#72 > store_cumulative#73))
-(68) TakeOrderedAndProject
-Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_cumulative#70, store_cumulative#71]
-Arguments: 100, [item_sk#54 ASC NULLS FIRST, d_date#55 ASC NULLS FIRST], [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_cumulative#70, store_cumulative#71]
+(70) TakeOrderedAndProject
+Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73]
+Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4
-BroadcastExchange (73)
-+- * Project (72)
- +- * Filter (71)
- +- * ColumnarToRow (70)
- +- Scan parquet default.date_dim (69)
+BroadcastExchange (75)
++- * Project (74)
+ +- * Filter (73)
+ +- * ColumnarToRow (72)
+ +- Scan parquet default.date_dim (71)
-(69) Scan parquet default.date_dim
-Output [3]: [d_date_sk#5, d_date#6, d_month_seq#72]
+(71) Scan parquet default.date_dim
+Output [3]: [d_date_sk#5, d_date#6, d_month_seq#74]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_date:date,d_month_seq:int>
-(70) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72]
+(72) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74]
-(71) Filter [codegen id : 1]
-Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72]
-Condition : (((isnotnull(d_month_seq#72) AND (d_month_seq#72 >= 1212)) AND (d_month_seq#72 <= 1223)) AND isnotnull(d_date_sk#5))
+(73) Filter [codegen id : 1]
+Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74]
+Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1212)) AND (d_month_seq#74 <= 1223)) AND isnotnull(d_date_sk#5))
-(72) Project [codegen id : 1]
+(74) Project [codegen id : 1]
Output [2]: [d_date_sk#5, d_date#6]
-Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72]
+Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74]
-(73) BroadcastExchange
+(75) BroadcastExchange
Input [2]: [d_date_sk#5, d_date#6]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75]
-Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#4
+Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#4
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt
index b1d245a..1a89b7c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt
@@ -1,5 +1,5 @@
TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative]
- WholeStageCodegen (58)
+ WholeStageCodegen (62)
Filter [web_cumulative,store_cumulative]
HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max]
HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max]
@@ -7,123 +7,129 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
SortMergeJoin [item_sk,item_sk,rk,rk]
InputAdapter
Window [item_sk,d_date]
- WholeStageCodegen (28)
+ WholeStageCodegen (30)
Sort [item_sk,d_date]
InputAdapter
Exchange [item_sk] #1
- WholeStageCodegen (27)
+ WholeStageCodegen (29)
Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales]
Filter [item_sk,item_sk]
SortMergeJoin [item_sk,d_date,item_sk,d_date]
InputAdapter
- WholeStageCodegen (13)
+ WholeStageCodegen (14)
Sort [item_sk,d_date]
- HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty]
- HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty]
- Project [item_sk,d_date,sumws,sumws]
- SortMergeJoin [item_sk,item_sk,rk,rk]
- InputAdapter
- WholeStageCodegen (6)
- Sort [item_sk]
+ InputAdapter
+ Exchange [item_sk,d_date] #2
+ WholeStageCodegen (13)
+ HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty]
+ HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty]
+ Project [item_sk,d_date,sumws,sumws]
+ SortMergeJoin [item_sk,item_sk,rk,rk]
InputAdapter
- Exchange [item_sk] #2
- WholeStageCodegen (5)
- Project [item_sk,d_date,sumws,rk]
- InputAdapter
- Window [ws_item_sk,d_date]
- WholeStageCodegen (4)
- Sort [ws_item_sk,d_date]
- InputAdapter
- Exchange [ws_item_sk] #3
- WholeStageCodegen (3)
- HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum]
- InputAdapter
- Exchange [ws_item_sk,d_date] #4
- WholeStageCodegen (2)
- HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum]
- Project [ws_item_sk,ws_sales_price,d_date]
- BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- Filter [ws_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #5
- WholeStageCodegen (1)
- Project [d_date_sk,d_date]
- Filter [d_month_seq,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq]
- InputAdapter
- ReusedExchange [d_date_sk,d_date] #5
- InputAdapter
- WholeStageCodegen (12)
- Sort [item_sk]
+ WholeStageCodegen (6)
+ Sort [item_sk]
+ InputAdapter
+ Exchange [item_sk] #3
+ WholeStageCodegen (5)
+ Project [item_sk,d_date,sumws,rk]
+ InputAdapter
+ Window [ws_item_sk,d_date]
+ WholeStageCodegen (4)
+ Sort [ws_item_sk,d_date]
+ InputAdapter
+ Exchange [ws_item_sk] #4
+ WholeStageCodegen (3)
+ HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum]
+ InputAdapter
+ Exchange [ws_item_sk,d_date] #5
+ WholeStageCodegen (2)
+ HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum]
+ Project [ws_item_sk,ws_sales_price,d_date]
+ BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ Filter [ws_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ Project [d_date_sk,d_date]
+ Filter [d_month_seq,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq]
+ InputAdapter
+ ReusedExchange [d_date_sk,d_date] #6
InputAdapter
- Exchange [item_sk] #6
- WholeStageCodegen (11)
- Project [item_sk,sumws,rk]
- InputAdapter
- Window [ws_item_sk,d_date]
- WholeStageCodegen (10)
- Sort [ws_item_sk,d_date]
- InputAdapter
- ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #3
+ WholeStageCodegen (12)
+ Sort [item_sk]
+ InputAdapter
+ Exchange [item_sk] #7
+ WholeStageCodegen (11)
+ Project [item_sk,sumws,rk]
+ InputAdapter
+ Window [ws_item_sk,d_date]
+ WholeStageCodegen (10)
+ Sort [ws_item_sk,d_date]
+ InputAdapter
+ ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4
InputAdapter
- WholeStageCodegen (26)
+ WholeStageCodegen (28)
Sort [item_sk,d_date]
- HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty]
- HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty]
- Project [item_sk,d_date,sumss,sumss]
- SortMergeJoin [item_sk,item_sk,rk,rk]
- InputAdapter
- WholeStageCodegen (19)
- Sort [item_sk]
+ InputAdapter
+ Exchange [item_sk,d_date] #8
+ WholeStageCodegen (27)
+ HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty]
+ HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty]
+ Project [item_sk,d_date,sumss,sumss]
+ SortMergeJoin [item_sk,item_sk,rk,rk]
InputAdapter
- Exchange [item_sk] #7
- WholeStageCodegen (18)
- Project [item_sk,d_date,sumss,rk]
- InputAdapter
- Window [ss_item_sk,d_date]
- WholeStageCodegen (17)
- Sort [ss_item_sk,d_date]
- InputAdapter
- Exchange [ss_item_sk] #8
- WholeStageCodegen (16)
- HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum]
- InputAdapter
- Exchange [ss_item_sk,d_date] #9
- WholeStageCodegen (15)
- HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum]
- Project [ss_item_sk,ss_sales_price,d_date]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Filter [ss_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
- ReusedSubquery [d_date_sk] #1
- InputAdapter
- ReusedExchange [d_date_sk,d_date] #5
- InputAdapter
- WholeStageCodegen (25)
- Sort [item_sk]
+ WholeStageCodegen (20)
+ Sort [item_sk]
+ InputAdapter
+ Exchange [item_sk] #9
+ WholeStageCodegen (19)
+ Project [item_sk,d_date,sumss,rk]
+ InputAdapter
+ Window [ss_item_sk,d_date]
+ WholeStageCodegen (18)
+ Sort [ss_item_sk,d_date]
+ InputAdapter
+ Exchange [ss_item_sk] #10
+ WholeStageCodegen (17)
+ HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum]
+ InputAdapter
+ Exchange [ss_item_sk,d_date] #11
+ WholeStageCodegen (16)
+ HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum]
+ Project [ss_item_sk,ss_sales_price,d_date]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Filter [ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ InputAdapter
+ ReusedExchange [d_date_sk,d_date] #6
InputAdapter
- Exchange [item_sk] #10
- WholeStageCodegen (24)
- Project [item_sk,sumss,rk]
- InputAdapter
- Window [ss_item_sk,d_date]
- WholeStageCodegen (23)
- Sort [ss_item_sk,d_date]
- InputAdapter
- ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #8
+ WholeStageCodegen (26)
+ Sort [item_sk]
+ InputAdapter
+ Exchange [item_sk] #12
+ WholeStageCodegen (25)
+ Project [item_sk,sumss,rk]
+ InputAdapter
+ Window [ss_item_sk,d_date]
+ WholeStageCodegen (24)
+ Sort [ss_item_sk,d_date]
+ InputAdapter
+ ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10
InputAdapter
- WholeStageCodegen (57)
+ WholeStageCodegen (61)
Project [item_sk,web_sales,store_sales,rk]
InputAdapter
Window [item_sk,d_date]
- WholeStageCodegen (56)
+ WholeStageCodegen (60)
Sort [item_sk,d_date]
InputAdapter
ReusedExchange [item_sk,d_date,web_sales,store_sales] #1
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
index aa9b899..d214b32 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
@@ -1,53 +1,56 @@
== Physical Plan ==
-TakeOrderedAndProject (49)
-+- * Project (48)
- +- * SortMergeJoin Inner (47)
- :- * Project (41)
- : +- * SortMergeJoin Inner (40)
- : :- * Sort (32)
- : : +- * Project (31)
- : : +- * Filter (30)
- : : +- Window (29)
- : : +- * Filter (28)
- : : +- Window (27)
- : : +- * Sort (26)
- : : +- Exchange (25)
- : : +- * HashAggregate (24)
- : : +- Exchange (23)
- : : +- * HashAggregate (22)
- : : +- * Project (21)
- : : +- * SortMergeJoin Inner (20)
- : : :- * Sort (14)
- : : : +- Exchange (13)
- : : : +- * Project (12)
- : : : +- * BroadcastHashJoin Inner BuildRight (11)
- : : : :- * Project (6)
- : : : : +- * BroadcastHashJoin Inner BuildRight (5)
- : : : : :- * Filter (3)
- : : : : : +- * ColumnarToRow (2)
- : : : : : +- Scan parquet default.catalog_sales (1)
- : : : : +- ReusedExchange (4)
- : : : +- BroadcastExchange (10)
- : : : +- * Filter (9)
- : : : +- * ColumnarToRow (8)
- : : : +- Scan parquet default.call_center (7)
- : : +- * Sort (19)
- : : +- Exchange (18)
- : : +- * Filter (17)
- : : +- * ColumnarToRow (16)
- : : +- Scan parquet default.item (15)
- : +- * Sort (39)
- : +- * Project (38)
- : +- Window (37)
- : +- * Sort (36)
- : +- Exchange (35)
- : +- * HashAggregate (34)
- : +- ReusedExchange (33)
- +- * Sort (46)
- +- * Project (45)
- +- Window (44)
- +- * Sort (43)
- +- ReusedExchange (42)
+TakeOrderedAndProject (52)
++- * Project (51)
+ +- * SortMergeJoin Inner (50)
+ :- * Project (43)
+ : +- * SortMergeJoin Inner (42)
+ : :- * Sort (33)
+ : : +- Exchange (32)
+ : : +- * Project (31)
+ : : +- * Filter (30)
+ : : +- Window (29)
+ : : +- * Filter (28)
+ : : +- Window (27)
+ : : +- * Sort (26)
+ : : +- Exchange (25)
+ : : +- * HashAggregate (24)
+ : : +- Exchange (23)
+ : : +- * HashAggregate (22)
+ : : +- * Project (21)
+ : : +- * SortMergeJoin Inner (20)
+ : : :- * Sort (14)
+ : : : +- Exchange (13)
+ : : : +- * Project (12)
+ : : : +- * BroadcastHashJoin Inner BuildRight (11)
+ : : : :- * Project (6)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (5)
+ : : : : :- * Filter (3)
+ : : : : : +- * ColumnarToRow (2)
+ : : : : : +- Scan parquet default.catalog_sales (1)
+ : : : : +- ReusedExchange (4)
+ : : : +- BroadcastExchange (10)
+ : : : +- * Filter (9)
+ : : : +- * ColumnarToRow (8)
+ : : : +- Scan parquet default.call_center (7)
+ : : +- * Sort (19)
+ : : +- Exchange (18)
+ : : +- * Filter (17)
+ : : +- * ColumnarToRow (16)
+ : : +- Scan parquet default.item (15)
+ : +- * Sort (41)
+ : +- Exchange (40)
+ : +- * Project (39)
+ : +- Window (38)
+ : +- * Sort (37)
+ : +- Exchange (36)
+ : +- * HashAggregate (35)
+ : +- ReusedExchange (34)
+ +- * Sort (49)
+ +- Exchange (48)
+ +- * Project (47)
+ +- Window (46)
+ +- * Sort (45)
+ +- ReusedExchange (44)
(1) Scan parquet default.catalog_sales
@@ -65,7 +68,7 @@ Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk
Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk#4]
Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_call_center_sk#1))
-(4) ReusedExchange [Reuses operator id: 53]
+(4) ReusedExchange [Reuses operator id: 56]
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
(5) BroadcastHashJoin [codegen id : 3]
@@ -189,106 +192,118 @@ Condition : ((isnotnull(avg_monthly_sales#25) AND (avg_monthly_sales#25 > 0.0000
Output [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24]
Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, _w0#22, rn#24, avg_monthly_sales#25]
-(32) Sort [codegen id : 11]
+(32) Exchange
+Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24]
+Arguments: hashpartitioning(i_category#15, i_brand#14, cc_name#10, rn#24, 5), ENSURE_REQUIREMENTS, [id=#26]
+
+(33) Sort [codegen id : 12]
Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24]
Arguments: [i_category#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST, rn#24 ASC NULLS FIRST], false, 0
-(33) ReusedExchange [Reuses operator id: 23]
-Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31]
+(34) ReusedExchange [Reuses operator id: 23]
+Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32]
-(34) HashAggregate [codegen id : 19]
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31]
-Keys [5]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30]
-Functions [1]: [sum(UnscaledValue(cs_sales_price#32))]
-Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#32))#20]
-Results [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#32))#20,17,2) AS sum_sales#21]
+(35) HashAggregate [codegen id : 20]
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32]
+Keys [5]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31]
+Functions [1]: [sum(UnscaledValue(cs_sales_price#33))]
+Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#20]
+Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#20,17,2) AS sum_sales#21]
-(35) Exchange
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21]
-Arguments: hashpartitioning(i_category#26, i_brand#27, cc_name#28, 5), ENSURE_REQUIREMENTS, [id=#33]
+(36) Exchange
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21]
+Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#34]
-(36) Sort [codegen id : 20]
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21]
-Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0
+(37) Sort [codegen id : 21]
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21]
+Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0
-(37) Window
-Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21]
-Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST]
+(38) Window
+Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21]
+Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST]
-(38) Project [codegen id : 21]
-Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#21 AS sum_sales#35, rn#34]
-Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21, rn#34]
+(39) Project [codegen id : 22]
+Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#21 AS sum_sales#36, rn#35]
+Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35]
-(39) Sort [codegen id : 21]
-Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34]
-Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, (rn#34 + 1) ASC NULLS FIRST], false, 0
+(40) Exchange
+Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35]
+Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#35 + 1), 5), ENSURE_REQUIREMENTS, [id=#37]
-(40) SortMergeJoin [codegen id : 22]
+(41) Sort [codegen id : 23]
+Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35]
+Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#35 + 1) ASC NULLS FIRST], false, 0
+
+(42) SortMergeJoin [codegen id : 24]
Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24]
-Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#34 + 1)]
+Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#35 + 1)]
Join condition: None
-(41) Project [codegen id : 22]
-Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35]
-Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34]
+(43) Project [codegen id : 24]
+Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36]
+Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35]
+
+(44) ReusedExchange [Reuses operator id: 36]
+Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21]
-(42) ReusedExchange [Reuses operator id: 35]
-Output [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21]
+(45) Sort [codegen id : 33]
+Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21]
+Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0
-(43) Sort [codegen id : 31]
-Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21]
-Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST], false, 0
+(46) Window
+Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21]
+Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST]
-(44) Window
-Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21]
-Arguments: [rank(d_year#39, d_moy#40) windowspecdefinition(i_category#36, i_brand#37, cc_name#38, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#36, i_brand#37, cc_name#38], [d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST]
+(47) Project [codegen id : 34]
+Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#21 AS sum_sales#44, rn#43]
+Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21, rn#43]
-(45) Project [codegen id : 32]
-Output [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#21 AS sum_sales#42, rn#41]
-Input [7]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21, rn#41]
+(48) Exchange
+Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43]
+Arguments: hashpartitioning(i_category#38, i_brand#39, cc_name#40, (rn#43 - 1), 5), ENSURE_REQUIREMENTS, [id=#45]
-(46) Sort [codegen id : 32]
-Input [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41]
-Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, (rn#41 - 1) ASC NULLS FIRST], false, 0
+(49) Sort [codegen id : 35]
+Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43]
+Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, (rn#43 - 1) ASC NULLS FIRST], false, 0
-(47) SortMergeJoin [codegen id : 33]
+(50) SortMergeJoin [codegen id : 36]
Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24]
-Right keys [4]: [i_category#36, i_brand#37, cc_name#38, (rn#41 - 1)]
+Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#43 - 1)]
Join condition: None
-(48) Project [codegen id : 33]
-Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#43, sum_sales#42 AS nsum#44]
-Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41]
+(51) Project [codegen id : 36]
+Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#46, sum_sales#44 AS nsum#47]
+Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43]
-(49) TakeOrderedAndProject
-Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44]
+(52) TakeOrderedAndProject
+Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
+Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (53)
-+- * Filter (52)
- +- * ColumnarToRow (51)
- +- Scan parquet default.date_dim (50)
+BroadcastExchange (56)
++- * Filter (55)
+ +- * ColumnarToRow (54)
+ +- Scan parquet default.date_dim (53)
-(50) Scan parquet default.date_dim
+(53) Scan parquet default.date_dim
Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
-(51) ColumnarToRow [codegen id : 1]
+(54) ColumnarToRow [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-(52) Filter [codegen id : 1]
+(55) Filter [codegen id : 1]
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6))
-(53) BroadcastExchange
+(56) BroadcastExchange
Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt
index 4389f60..b464f55 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt
@@ -1,95 +1,104 @@
TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum]
- WholeStageCodegen (33)
+ WholeStageCodegen (36)
Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales]
SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn]
InputAdapter
- WholeStageCodegen (22)
+ WholeStageCodegen (24)
Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales]
SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn]
InputAdapter
- WholeStageCodegen (11)
+ WholeStageCodegen (12)
Sort [i_category,i_brand,cc_name,rn]
- Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
- Filter [avg_monthly_sales,sum_sales]
- InputAdapter
- Window [_w0,i_category,i_brand,cc_name,d_year]
- WholeStageCodegen (10)
- Filter [d_year]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,cc_name]
- WholeStageCodegen (9)
- Sort [i_category,i_brand,cc_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,cc_name] #1
- WholeStageCodegen (8)
- HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum]
- InputAdapter
- Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2
- WholeStageCodegen (7)
- HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum]
- Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- SortMergeJoin [cs_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (4)
- Sort [cs_item_sk]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,rn] #1
+ WholeStageCodegen (11)
+ Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn]
+ Filter [avg_monthly_sales,sum_sales]
+ InputAdapter
+ Window [_w0,i_category,i_brand,cc_name,d_year]
+ WholeStageCodegen (10)
+ Filter [d_year]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,cc_name]
+ WholeStageCodegen (9)
+ Sort [i_category,i_brand,cc_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name] #2
+ WholeStageCodegen (8)
+ HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3
+ WholeStageCodegen (7)
+ HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum]
+ Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
+ SortMergeJoin [cs_item_sk,i_item_sk]
InputAdapter
- Exchange [cs_item_sk] #3
- WholeStageCodegen (3)
- Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name]
- BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
- Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy]
- BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- Filter [cs_item_sk,cs_call_center_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #4
- WholeStageCodegen (1)
- Filter [d_year,d_moy,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
- InputAdapter
- ReusedExchange [d_date_sk,d_year,d_moy] #4
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (2)
- Filter [cc_call_center_sk,cc_name]
- ColumnarToRow
+ WholeStageCodegen (4)
+ Sort [cs_item_sk]
+ InputAdapter
+ Exchange [cs_item_sk] #4
+ WholeStageCodegen (3)
+ Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name]
+ BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ Filter [cs_item_sk,cs_call_center_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ Filter [d_year,d_moy,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
- Scan parquet default.call_center [cc_call_center_sk,cc_name]
- InputAdapter
- WholeStageCodegen (6)
- Sort [i_item_sk]
+ ReusedExchange [d_date_sk,d_year,d_moy] #5
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ Filter [cc_call_center_sk,cc_name]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.call_center [cc_call_center_sk,cc_name]
InputAdapter
- Exchange [i_item_sk] #6
- WholeStageCodegen (5)
- Filter [i_item_sk,i_category,i_brand]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_brand,i_category]
+ WholeStageCodegen (6)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #7
+ WholeStageCodegen (5)
+ Filter [i_item_sk,i_category,i_brand]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_brand,i_category]
InputAdapter
- WholeStageCodegen (21)
+ WholeStageCodegen (23)
Sort [i_category,i_brand,cc_name,rn]
- Project [i_category,i_brand,cc_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,cc_name]
- WholeStageCodegen (20)
- Sort [i_category,i_brand,cc_name,d_year,d_moy]
- InputAdapter
- Exchange [i_category,i_brand,cc_name] #7
- WholeStageCodegen (19)
- HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum]
- InputAdapter
- ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,rn] #8
+ WholeStageCodegen (22)
+ Project [i_category,i_brand,cc_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,cc_name]
+ WholeStageCodegen (21)
+ Sort [i_category,i_brand,cc_name,d_year,d_moy]
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name] #9
+ WholeStageCodegen (20)
+ HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3
InputAdapter
- WholeStageCodegen (32)
+ WholeStageCodegen (35)
Sort [i_category,i_brand,cc_name,rn]
- Project [i_category,i_brand,cc_name,sum_sales,rn]
- InputAdapter
- Window [d_year,d_moy,i_category,i_brand,cc_name]
- WholeStageCodegen (31)
- Sort [i_category,i_brand,cc_name,d_year,d_moy]
- InputAdapter
- ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7
+ InputAdapter
+ Exchange [i_category,i_brand,cc_name,rn] #10
+ WholeStageCodegen (34)
+ Project [i_category,i_brand,cc_name,sum_sales,rn]
+ InputAdapter
+ Window [d_year,d_moy,i_category,i_brand,cc_name]
+ WholeStageCodegen (33)
+ Sort [i_category,i_brand,cc_name,d_year,d_moy]
+ InputAdapter
+ ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #9
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 cfee229..ddaa34a 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
@@ -1,185 +1,187 @@
== Physical Plan ==
-* Sort (181)
-+- Exchange (180)
- +- * Project (179)
- +- * SortMergeJoin Inner (178)
- :- * Sort (110)
- : +- * HashAggregate (109)
- : +- * HashAggregate (108)
- : +- * Project (107)
- : +- * BroadcastHashJoin Inner BuildRight (106)
- : :- * Project (100)
- : : +- * BroadcastHashJoin Inner BuildRight (99)
- : : :- * Project (97)
- : : : +- * BroadcastHashJoin Inner BuildRight (96)
- : : : :- * Project (91)
- : : : : +- * BroadcastHashJoin Inner BuildRight (90)
- : : : : :- * Project (88)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (87)
- : : : : : :- * Project (82)
- : : : : : : +- * BroadcastHashJoin Inner BuildRight (81)
- : : : : : : :- * Project (79)
- : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78)
- : : : : : : : :- * Project (73)
- : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72)
- : : : : : : : : :- * Project (67)
- : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66)
- : : : : : : : : : :- * Project (64)
- : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63)
- : : : : : : : : : : :- * Project (58)
- : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57)
- : : : : : : : : : : : :- * Project (55)
- : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54)
- : : : : : : : : : : : : :- * Project (49)
- : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48)
- : : : : : : : : : : : : : :- * Project (43)
- : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42)
- : : : : : : : : : : : : : : :- * Project (37)
- : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36)
- : : : : : : : : : : : : : : : :- * Project (34)
- : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (33)
- : : : : : : : : : : : : : : : : :- * Sort (12)
- : : : : : : : : : : : : : : : : : +- Exchange (11)
- : : : : : : : : : : : : : : : : : +- * Project (10)
- : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9)
- : : : : : : : : : : : : : : : : : :- BroadcastExchange (4)
- : : : : : : : : : : : : : : : : : : +- * Filter (3)
- : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2)
- : : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (1)
- : : : : : : : : : : : : : : : : : +- * Project (8)
- : : : : : : : : : : : : : : : : : +- * Filter (7)
- : : : : : : : : : : : : : : : : : +- * ColumnarToRow (6)
- : : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (5)
- : : : : : : : : : : : : : : : : +- * Sort (32)
- : : : : : : : : : : : : : : : : +- * Project (31)
- : : : : : : : : : : : : : : : : +- * Filter (30)
- : : : : : : : : : : : : : : : : +- * HashAggregate (29)
- : : : : : : : : : : : : : : : : +- Exchange (28)
- : : : : : : : : : : : : : : : : +- * HashAggregate (27)
- : : : : : : : : : : : : : : : : +- * Project (26)
- : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (25)
- : : : : : : : : : : : : : : : : :- * Sort (18)
- : : : : : : : : : : : : : : : : : +- Exchange (17)
- : : : : : : : : : : : : : : : : : +- * Project (16)
- : : : : : : : : : : : : : : : : : +- * Filter (15)
- : : : : : : : : : : : : : : : : : +- * ColumnarToRow (14)
- : : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_sales (13)
- : : : : : : : : : : : : : : : : +- * Sort (24)
- : : : : : : : : : : : : : : : : +- Exchange (23)
- : : : : : : : : : : : : : : : : +- * Project (22)
- : : : : : : : : : : : : : : : : +- * Filter (21)
- : : : : : : : : : : : : : : : : +- * ColumnarToRow (20)
- : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_returns (19)
- : : : : : : : : : : : : : : : +- ReusedExchange (35)
- : : : : : : : : : : : : : : +- BroadcastExchange (41)
- : : : : : : : : : : : : : : +- * Filter (40)
- : : : : : : : : : : : : : : +- * ColumnarToRow (39)
- : : : : : : : : : : : : : : +- Scan parquet default.store (38)
- : : : : : : : : : : : : : +- BroadcastExchange (47)
- : : : : : : : : : : : : : +- * Filter (46)
- : : : : : : : : : : : : : +- * ColumnarToRow (45)
- : : : : : : : : : : : : : +- Scan parquet default.customer (44)
- : : : : : : : : : : : : +- BroadcastExchange (53)
- : : : : : : : : : : : : +- * Filter (52)
- : : : : : : : : : : : : +- * ColumnarToRow (51)
- : : : : : : : : : : : : +- Scan parquet default.date_dim (50)
- : : : : : : : : : : : +- ReusedExchange (56)
- : : : : : : : : : : +- BroadcastExchange (62)
- : : : : : : : : : : +- * Filter (61)
- : : : : : : : : : : +- * ColumnarToRow (60)
- : : : : : : : : : : +- Scan parquet default.customer_demographics (59)
- : : : : : : : : : +- ReusedExchange (65)
- : : : : : : : : +- BroadcastExchange (71)
- : : : : : : : : +- * Filter (70)
- : : : : : : : : +- * ColumnarToRow (69)
- : : : : : : : : +- Scan parquet default.promotion (68)
- : : : : : : : +- BroadcastExchange (77)
- : : : : : : : +- * Filter (76)
- : : : : : : : +- * ColumnarToRow (75)
- : : : : : : : +- Scan parquet default.household_demographics (74)
- : : : : : : +- ReusedExchange (80)
- : : : : : +- BroadcastExchange (86)
- : : : : : +- * Filter (85)
- : : : : : +- * ColumnarToRow (84)
- : : : : : +- Scan parquet default.customer_address (83)
- : : : : +- ReusedExchange (89)
- : : : +- BroadcastExchange (95)
- : : : +- * Filter (94)
- : : : +- * ColumnarToRow (93)
- : : : +- Scan parquet default.income_band (92)
- : : +- ReusedExchange (98)
- : +- BroadcastExchange (105)
- : +- * Project (104)
- : +- * Filter (103)
- : +- * ColumnarToRow (102)
- : +- Scan parquet default.item (101)
- +- * Sort (177)
- +- * HashAggregate (176)
- +- * HashAggregate (175)
- +- * Project (174)
- +- * BroadcastHashJoin Inner BuildRight (173)
- :- * Project (171)
- : +- * BroadcastHashJoin Inner BuildRight (170)
- : :- * Project (168)
- : : +- * BroadcastHashJoin Inner BuildRight (167)
- : : :- * Project (165)
- : : : +- * BroadcastHashJoin Inner BuildRight (164)
- : : : :- * Project (162)
- : : : : +- * BroadcastHashJoin Inner BuildRight (161)
- : : : : :- * Project (159)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (158)
- : : : : : :- * Project (156)
- : : : : : : +- * BroadcastHashJoin Inner BuildRight (155)
- : : : : : : :- * Project (153)
- : : : : : : : +- * BroadcastHashJoin Inner BuildRight (152)
- : : : : : : : :- * Project (150)
- : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149)
- : : : : : : : : :- * Project (147)
- : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146)
- : : : : : : : : : :- * Project (144)
- : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143)
- : : : : : : : : : : :- * Project (141)
- : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140)
- : : : : : : : : : : : :- * Project (138)
- : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137)
- : : : : : : : : : : : : :- * Project (135)
- : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134)
- : : : : : : : : : : : : : :- * Project (132)
- : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131)
- : : : : : : : : : : : : : : :- * Project (129)
- : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (128)
- : : : : : : : : : : : : : : : :- * Sort (122)
- : : : : : : : : : : : : : : : : +- Exchange (121)
- : : : : : : : : : : : : : : : : +- * Project (120)
- : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (119)
- : : : : : : : : : : : : : : : : :- BroadcastExchange (114)
- : : : : : : : : : : : : : : : : : +- * Filter (113)
- : : : : : : : : : : : : : : : : : +- * ColumnarToRow (112)
- : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (111)
- : : : : : : : : : : : : : : : : +- * Project (118)
- : : : : : : : : : : : : : : : : +- * Filter (117)
- : : : : : : : : : : : : : : : : +- * ColumnarToRow (116)
- : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (115)
- : : : : : : : : : : : : : : : +- * Sort (127)
- : : : : : : : : : : : : : : : +- * Project (126)
- : : : : : : : : : : : : : : : +- * Filter (125)
- : : : : : : : : : : : : : : : +- * HashAggregate (124)
- : : : : : : : : : : : : : : : +- ReusedExchange (123)
- : : : : : : : : : : : : : : +- ReusedExchange (130)
- : : : : : : : : : : : : : +- ReusedExchange (133)
- : : : : : : : : : : : : +- ReusedExchange (136)
- : : : : : : : : : : : +- ReusedExchange (139)
- : : : : : : : : : : +- ReusedExchange (142)
- : : : : : : : : : +- ReusedExchange (145)
- : : : : : : : : +- ReusedExchange (148)
- : : : : : : : +- ReusedExchange (151)
- : : : : : : +- ReusedExchange (154)
- : : : : : +- ReusedExchange (157)
- : : : : +- ReusedExchange (160)
- : : : +- ReusedExchange (163)
- : : +- ReusedExchange (166)
- : +- ReusedExchange (169)
- +- ReusedExchange (172)
+* Sort (183)
++- Exchange (182)
+ +- * Project (181)
+ +- * SortMergeJoin Inner (180)
+ :- * Sort (111)
+ : +- Exchange (110)
+ : +- * HashAggregate (109)
+ : +- * HashAggregate (108)
+ : +- * Project (107)
+ : +- * BroadcastHashJoin Inner BuildRight (106)
+ : :- * Project (100)
+ : : +- * BroadcastHashJoin Inner BuildRight (99)
+ : : :- * Project (97)
+ : : : +- * BroadcastHashJoin Inner BuildRight (96)
+ : : : :- * Project (91)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (90)
+ : : : : :- * Project (88)
+ : : : : : +- * BroadcastHashJoin Inner BuildRight (87)
+ : : : : : :- * Project (82)
+ : : : : : : +- * BroadcastHashJoin Inner BuildRight (81)
+ : : : : : : :- * Project (79)
+ : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78)
+ : : : : : : : :- * Project (73)
+ : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72)
+ : : : : : : : : :- * Project (67)
+ : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66)
+ : : : : : : : : : :- * Project (64)
+ : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63)
+ : : : : : : : : : : :- * Project (58)
+ : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57)
+ : : : : : : : : : : : :- * Project (55)
+ : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54)
+ : : : : : : : : : : : : :- * Project (49)
+ : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48)
+ : : : : : : : : : : : : : :- * Project (43)
+ : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42)
+ : : : : : : : : : : : : : : :- * Project (37)
+ : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36)
+ : : : : : : : : : : : : : : : :- * Project (34)
+ : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (33)
+ : : : : : : : : : : : : : : : : :- * Sort (12)
+ : : : : : : : : : : : : : : : : : +- Exchange (11)
+ : : : : : : : : : : : : : : : : : +- * Project (10)
+ : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9)
+ : : : : : : : : : : : : : : : : : :- BroadcastExchange (4)
+ : : : : : : : : : : : : : : : : : : +- * Filter (3)
+ : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2)
+ : : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (1)
+ : : : : : : : : : : : : : : : : : +- * Project (8)
+ : : : : : : : : : : : : : : : : : +- * Filter (7)
+ : : : : : : : : : : : : : : : : : +- * ColumnarToRow (6)
+ : : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (5)
+ : : : : : : : : : : : : : : : : +- * Sort (32)
+ : : : : : : : : : : : : : : : : +- * Project (31)
+ : : : : : : : : : : : : : : : : +- * Filter (30)
+ : : : : : : : : : : : : : : : : +- * HashAggregate (29)
+ : : : : : : : : : : : : : : : : +- Exchange (28)
+ : : : : : : : : : : : : : : : : +- * HashAggregate (27)
+ : : : : : : : : : : : : : : : : +- * Project (26)
+ : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (25)
+ : : : : : : : : : : : : : : : : :- * Sort (18)
+ : : : : : : : : : : : : : : : : : +- Exchange (17)
+ : : : : : : : : : : : : : : : : : +- * Project (16)
+ : : : : : : : : : : : : : : : : : +- * Filter (15)
+ : : : : : : : : : : : : : : : : : +- * ColumnarToRow (14)
+ : : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_sales (13)
+ : : : : : : : : : : : : : : : : +- * Sort (24)
+ : : : : : : : : : : : : : : : : +- Exchange (23)
+ : : : : : : : : : : : : : : : : +- * Project (22)
+ : : : : : : : : : : : : : : : : +- * Filter (21)
+ : : : : : : : : : : : : : : : : +- * ColumnarToRow (20)
+ : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_returns (19)
+ : : : : : : : : : : : : : : : +- ReusedExchange (35)
+ : : : : : : : : : : : : : : +- BroadcastExchange (41)
+ : : : : : : : : : : : : : : +- * Filter (40)
+ : : : : : : : : : : : : : : +- * ColumnarToRow (39)
+ : : : : : : : : : : : : : : +- Scan parquet default.store (38)
+ : : : : : : : : : : : : : +- BroadcastExchange (47)
+ : : : : : : : : : : : : : +- * Filter (46)
+ : : : : : : : : : : : : : +- * ColumnarToRow (45)
+ : : : : : : : : : : : : : +- Scan parquet default.customer (44)
+ : : : : : : : : : : : : +- BroadcastExchange (53)
+ : : : : : : : : : : : : +- * Filter (52)
+ : : : : : : : : : : : : +- * ColumnarToRow (51)
+ : : : : : : : : : : : : +- Scan parquet default.date_dim (50)
+ : : : : : : : : : : : +- ReusedExchange (56)
+ : : : : : : : : : : +- BroadcastExchange (62)
+ : : : : : : : : : : +- * Filter (61)
+ : : : : : : : : : : +- * ColumnarToRow (60)
+ : : : : : : : : : : +- Scan parquet default.customer_demographics (59)
+ : : : : : : : : : +- ReusedExchange (65)
+ : : : : : : : : +- BroadcastExchange (71)
+ : : : : : : : : +- * Filter (70)
+ : : : : : : : : +- * ColumnarToRow (69)
+ : : : : : : : : +- Scan parquet default.promotion (68)
+ : : : : : : : +- BroadcastExchange (77)
+ : : : : : : : +- * Filter (76)
+ : : : : : : : +- * ColumnarToRow (75)
+ : : : : : : : +- Scan parquet default.household_demographics (74)
+ : : : : : : +- ReusedExchange (80)
+ : : : : : +- BroadcastExchange (86)
+ : : : : : +- * Filter (85)
+ : : : : : +- * ColumnarToRow (84)
+ : : : : : +- Scan parquet default.customer_address (83)
+ : : : : +- ReusedExchange (89)
+ : : : +- BroadcastExchange (95)
+ : : : +- * Filter (94)
+ : : : +- * ColumnarToRow (93)
+ : : : +- Scan parquet default.income_band (92)
+ : : +- ReusedExchange (98)
+ : +- BroadcastExchange (105)
+ : +- * Project (104)
+ : +- * Filter (103)
+ : +- * ColumnarToRow (102)
+ : +- Scan parquet default.item (101)
+ +- * Sort (179)
+ +- Exchange (178)
+ +- * HashAggregate (177)
+ +- * HashAggregate (176)
+ +- * Project (175)
+ +- * BroadcastHashJoin Inner BuildRight (174)
+ :- * Project (172)
+ : +- * BroadcastHashJoin Inner BuildRight (171)
+ : :- * Project (169)
+ : : +- * BroadcastHashJoin Inner BuildRight (168)
+ : : :- * Project (166)
+ : : : +- * BroadcastHashJoin Inner BuildRight (165)
+ : : : :- * Project (163)
+ : : : : +- * BroadcastHashJoin Inner BuildRight (162)
+ : : : : :- * Project (160)
+ : : : : : +- * BroadcastHashJoin Inner BuildRight (159)
+ : : : : : :- * Project (157)
+ : : : : : : +- * BroadcastHashJoin Inner BuildRight (156)
+ : : : : : : :- * Project (154)
+ : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153)
+ : : : : : : : :- * Project (151)
+ : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150)
+ : : : : : : : : :- * Project (148)
+ : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147)
+ : : : : : : : : : :- * Project (145)
+ : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144)
+ : : : : : : : : : : :- * Project (142)
+ : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141)
+ : : : : : : : : : : : :- * Project (139)
+ : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138)
+ : : : : : : : : : : : : :- * Project (136)
+ : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135)
+ : : : : : : : : : : : : : :- * Project (133)
+ : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (132)
+ : : : : : : : : : : : : : : :- * Project (130)
+ : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (129)
+ : : : : : : : : : : : : : : : :- * Sort (123)
+ : : : : : : : : : : : : : : : : +- Exchange (122)
+ : : : : : : : : : : : : : : : : +- * Project (121)
+ : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (120)
+ : : : : : : : : : : : : : : : : :- BroadcastExchange (115)
+ : : : : : : : : : : : : : : : : : +- * Filter (114)
+ : : : : : : : : : : : : : : : : : +- * ColumnarToRow (113)
+ : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (112)
+ : : : : : : : : : : : : : : : : +- * Project (119)
+ : : : : : : : : : : : : : : : : +- * Filter (118)
+ : : : : : : : : : : : : : : : : +- * ColumnarToRow (117)
+ : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (116)
+ : : : : : : : : : : : : : : : +- * Sort (128)
+ : : : : : : : : : : : : : : : +- * Project (127)
+ : : : : : : : : : : : : : : : +- * Filter (126)
+ : : : : : : : : : : : : : : : +- * HashAggregate (125)
+ : : : : : : : : : : : : : : : +- ReusedExchange (124)
+ : : : : : : : : : : : : : : +- ReusedExchange (131)
+ : : : : : : : : : : : : : +- ReusedExchange (134)
+ : : : : : : : : : : : : +- ReusedExchange (137)
+ : : : : : : : : : : : +- ReusedExchange (140)
+ : : : : : : : : : : +- ReusedExchange (143)
+ : : : : : : : : : +- ReusedExchange (146)
+ : : : : : : : : +- ReusedExchange (149)
+ : : : : : : : +- ReusedExchange (152)
+ : : : : : : +- ReusedExchange (155)
+ : : : : : +- ReusedExchange (158)
+ : : : : +- ReusedExchange (161)
+ : : : +- ReusedExchange (164)
+ : : +- ReusedExchange (167)
+ : +- ReusedExchange (170)
+ +- ReusedExchange (173)
(1) Scan parquet default.store_sales
@@ -336,7 +338,7 @@ Join condition: None
Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12]
Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19]
-(35) ReusedExchange [Reuses operator id: 185]
+(35) ReusedExchange [Reuses operator id: 187]
Output [2]: [d_date_sk#42, d_year#43]
(36) BroadcastHashJoin [codegen id : 25]
@@ -669,360 +671,368 @@ Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledV
Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#9))#100, sum(UnscaledValue(ss_list_price#10))#101, sum(UnscaledValue(ss_coupon_amt#11))#102]
Results [17]: [i_product_name#89 AS product_name#103, i_item_sk#86 AS item_sk#104, s_store_name#45 AS store_name#105, s_zip#46 AS store_zip#106, ca_street_number#73 AS b_street_number#107, ca_street_name#74 AS b_streen_name#108, ca_city#75 AS b_city#109, ca_zip#76 AS b_zip#110, ca_street_number#79 AS c_street_number#111, ca_street_name#80 AS c_street_name#112, ca_city#81 AS c_city#113, ca_zip#82 AS c_zip#114, d_year#43 AS syear#115, count(1)#99 AS cnt#116, MakeDecimal(sum(UnscaledValue(s [...]
-(110) Sort [codegen id : 25]
+(110) Exchange
+Input [17]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119]
+Arguments: hashpartitioning(item_sk#104, store_name#105, store_zip#106, 5), ENSURE_REQUIREMENTS, [id=#120]
+
+(111) Sort [codegen id : 26]
Input [17]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119]
Arguments: [item_sk#104 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, store_zip#106 ASC NULLS FIRST], false, 0
-(111) Scan parquet default.store_sales
-Output [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
+(112) Scan parquet default.store_sales
+Output [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)]
+PartitionFilters: [isnotnull(ss_sold_date_sk#132), dynamicpruningexpression(ss_sold_date_sk#132 IN dynamicpruning#133)]
PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)]
ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_cdemo_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)>
-(112) ColumnarToRow [codegen id : 26]
-Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
+(113) ColumnarToRow [codegen id : 27]
+Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
-(113) Filter [codegen id : 26]
-Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
-Condition : (((((((isnotnull(ss_item_sk#120) AND isnotnull(ss_ticket_number#127)) AND isnotnull(ss_store_sk#125)) AND isnotnull(ss_customer_sk#121)) AND isnotnull(ss_cdemo_sk#122)) AND isnotnull(ss_promo_sk#126)) AND isnotnull(ss_hdemo_sk#123)) AND isnotnull(ss_addr_sk#124))
+(114) Filter [codegen id : 27]
+Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
+Condition : (((((((isnotnull(ss_item_sk#121) AND isnotnull(ss_ticket_number#128)) AND isnotnull(ss_store_sk#126)) AND isnotnull(ss_customer_sk#122)) AND isnotnull(ss_cdemo_sk#123)) AND isnotnull(ss_promo_sk#127)) AND isnotnull(ss_hdemo_sk#124)) AND isnotnull(ss_addr_sk#125))
-(114) BroadcastExchange
-Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
-Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#133]
+(115) BroadcastExchange
+Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
+Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#134]
-(115) Scan parquet default.store_returns
-Output [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136]
+(116) Scan parquet default.store_returns
+Output [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137]
Batched: true
Location [not included in comparison]/{warehouse_dir}/store_returns]
PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int>
-(116) ColumnarToRow
-Input [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136]
+(117) ColumnarToRow
+Input [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137]
-(117) Filter
-Input [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136]
-Condition : (isnotnull(sr_item_sk#134) AND isnotnull(sr_ticket_number#135))
+(118) Filter
+Input [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137]
+Condition : (isnotnull(sr_item_sk#135) AND isnotnull(sr_ticket_number#136))
-(118) Project
-Output [2]: [sr_item_sk#134, sr_ticket_number#135]
-Input [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136]
+(119) Project
+Output [2]: [sr_item_sk#135, sr_ticket_number#136]
+Input [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137]
-(119) BroadcastHashJoin [codegen id : 27]
-Left keys [2]: [ss_item_sk#120, ss_ticket_number#127]
-Right keys [2]: [sr_item_sk#134, sr_ticket_number#135]
+(120) BroadcastHashJoin [codegen id : 28]
+Left keys [2]: [ss_item_sk#121, ss_ticket_number#128]
+Right keys [2]: [sr_item_sk#135, sr_ticket_number#136]
Join condition: None
-(120) Project [codegen id : 27]
-Output [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
-Input [14]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131, sr_item_sk#134, sr_ticket_number#135]
+(121) Project [codegen id : 28]
+Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
+Input [14]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, sr_item_sk#135, sr_ticket_number#136]
-(121) Exchange
-Input [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
-Arguments: hashpartitioning(ss_item_sk#120, 5), ENSURE_REQUIREMENTS, [id=#137]
+(122) Exchange
+Input [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
+Arguments: hashpartitioning(ss_item_sk#121, 5), ENSURE_REQUIREMENTS, [id=#138]
-(122) Sort [codegen id : 28]
-Input [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
-Arguments: [ss_item_sk#120 ASC NULLS FIRST], false, 0
+(123) Sort [codegen id : 29]
+Input [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
+Arguments: [ss_item_sk#121 ASC NULLS FIRST], false, 0
-(123) ReusedExchange [Reuses operator id: 28]
-Output [4]: [cs_item_sk#138, sum#139, sum#140, isEmpty#141]
+(124) ReusedExchange [Reuses operator id: 28]
+Output [4]: [cs_item_sk#139, sum#140, sum#141, isEmpty#142]
-(124) HashAggregate [codegen id : 34]
-Input [4]: [cs_item_sk#138, sum#139, sum#140, isEmpty#141]
-Keys [1]: [cs_item_sk#138]
-Functions [2]: [sum(UnscaledValue(cs_ext_list_price#142)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#143 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#144 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#145 as decimal(9,2)))), DecimalType(9,2), true))]
-Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#142))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#143 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#144 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#145 as decimal(9,2)))), DecimalType(9,2), true))#39]
-Results [3]: [cs_item_sk#138, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#142))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#143 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#144 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#145 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41]
+(125) HashAggregate [codegen id : 35]
+Input [4]: [cs_item_sk#139, sum#140, sum#141, isEmpty#142]
+Keys [1]: [cs_item_sk#139]
+Functions [2]: [sum(UnscaledValue(cs_ext_list_price#143)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))]
+Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#143))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#39]
+Results [3]: [cs_item_sk#139, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#143))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41]
-(125) Filter [codegen id : 34]
-Input [3]: [cs_item_sk#138, sale#40, refund#41]
+(126) Filter [codegen id : 35]
+Input [3]: [cs_item_sk#139, sale#40, refund#41]
Condition : (isnotnull(sale#40) AND (cast(sale#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#41)), DecimalType(21,2), true)))
-(126) Project [codegen id : 34]
-Output [1]: [cs_item_sk#138]
-Input [3]: [cs_item_sk#138, sale#40, refund#41]
+(127) Project [codegen id : 35]
+Output [1]: [cs_item_sk#139]
+Input [3]: [cs_item_sk#139, sale#40, refund#41]
-(127) Sort [codegen id : 34]
-Input [1]: [cs_item_sk#138]
-Arguments: [cs_item_sk#138 ASC NULLS FIRST], false, 0
+(128) Sort [codegen id : 35]
+Input [1]: [cs_item_sk#139]
+Arguments: [cs_item_sk#139 ASC NULLS FIRST], false, 0
-(128) SortMergeJoin [codegen id : 50]
-Left keys [1]: [ss_item_sk#120]
-Right keys [1]: [cs_item_sk#138]
+(129) SortMergeJoin [codegen id : 51]
+Left keys [1]: [ss_item_sk#121]
+Right keys [1]: [cs_item_sk#139]
Join condition: None
-(129) Project [codegen id : 50]
-Output [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131]
-Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131, cs_item_sk#138]
+(130) Project [codegen id : 51]
+Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132]
+Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, cs_item_sk#139]
-(130) ReusedExchange [Reuses operator id: 189]
-Output [2]: [d_date_sk#146, d_year#147]
+(131) ReusedExchange [Reuses operator id: 191]
+Output [2]: [d_date_sk#147, d_year#148]
-(131) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_sold_date_sk#131]
-Right keys [1]: [d_date_sk#146]
+(132) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_sold_date_sk#132]
+Right keys [1]: [d_date_sk#147]
Join condition: None
-(132) Project [codegen id : 50]
-Output [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147]
-Input [13]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131, d_date_sk#146, d_year#147]
+(133) Project [codegen id : 51]
+Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148]
+Input [13]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, d_date_sk#147, d_year#148]
-(133) ReusedExchange [Reuses operator id: 41]
-Output [3]: [s_store_sk#148, s_store_name#149, s_zip#150]
+(134) ReusedExchange [Reuses operator id: 41]
+Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151]
-(134) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_store_sk#125]
-Right keys [1]: [s_store_sk#148]
+(135) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_store_sk#126]
+Right keys [1]: [s_store_sk#149]
Join condition: None
-(135) Project [codegen id : 50]
-Output [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150]
-Input [14]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_sk#148, s_store_name#149, s_zip#150]
+(136) Project [codegen id : 51]
+Output [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151]
+Input [14]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_sk#149, s_store_name#150, s_zip#151]
-(136) ReusedExchange [Reuses operator id: 47]
-Output [6]: [c_customer_sk#151, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156]
+(137) ReusedExchange [Reuses operator id: 47]
+Output [6]: [c_customer_sk#152, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157]
-(137) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_customer_sk#121]
-Right keys [1]: [c_customer_sk#151]
+(138) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_customer_sk#122]
+Right keys [1]: [c_customer_sk#152]
Join condition: None
-(138) Project [codegen id : 50]
-Output [16]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156]
-Input [18]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_customer_sk#151, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156]
+(139) Project [codegen id : 51]
+Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157]
+Input [18]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_customer_sk#152, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157]
-(139) ReusedExchange [Reuses operator id: 53]
-Output [2]: [d_date_sk#157, d_year#158]
+(140) ReusedExchange [Reuses operator id: 53]
+Output [2]: [d_date_sk#158, d_year#159]
-(140) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [c_first_sales_date_sk#156]
-Right keys [1]: [d_date_sk#157]
+(141) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [c_first_sales_date_sk#157]
+Right keys [1]: [d_date_sk#158]
Join condition: None
-(141) Project [codegen id : 50]
-Output [16]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, d_year#158]
-Input [18]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156, d_date_sk#157, d_year#158]
+(142) Project [codegen id : 51]
+Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, d_year#159]
+Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157, d_date_sk#158, d_year#159]
-(142) ReusedExchange [Reuses operator id: 53]
-Output [2]: [d_date_sk#159, d_year#160]
+(143) ReusedExchange [Reuses operator id: 53]
+Output [2]: [d_date_sk#160, d_year#161]
-(143) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [c_first_shipto_date_sk#155]
-Right keys [1]: [d_date_sk#159]
+(144) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [c_first_shipto_date_sk#156]
+Right keys [1]: [d_date_sk#160]
Join condition: None
-(144) Project [codegen id : 50]
-Output [16]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160]
-Input [18]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, d_year#158, d_date_sk#159, d_year#160]
+(145) Project [codegen id : 51]
+Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161]
+Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, d_year#159, d_date_sk#160, d_year#161]
-(145) ReusedExchange [Reuses operator id: 62]
-Output [2]: [cd_demo_sk#161, cd_marital_status#162]
+(146) ReusedExchange [Reuses operator id: 62]
+Output [2]: [cd_demo_sk#162, cd_marital_status#163]
-(146) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_cdemo_sk#122]
-Right keys [1]: [cd_demo_sk#161]
+(147) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_cdemo_sk#123]
+Right keys [1]: [cd_demo_sk#162]
Join condition: None
-(147) Project [codegen id : 50]
-Output [16]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, cd_marital_status#162]
-Input [18]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, cd_demo_sk#161, cd_marital_status#162]
+(148) Project [codegen id : 51]
+Output [16]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_marital_status#163]
+Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_demo_sk#162, cd_marital_status#163]
-(148) ReusedExchange [Reuses operator id: 62]
-Output [2]: [cd_demo_sk#163, cd_marital_status#164]
+(149) ReusedExchange [Reuses operator id: 62]
+Output [2]: [cd_demo_sk#164, cd_marital_status#165]
-(149) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [c_current_cdemo_sk#152]
-Right keys [1]: [cd_demo_sk#163]
-Join condition: NOT (cd_marital_status#162 = cd_marital_status#164)
+(150) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [c_current_cdemo_sk#153]
+Right keys [1]: [cd_demo_sk#164]
+Join condition: NOT (cd_marital_status#163 = cd_marital_status#165)
-(150) Project [codegen id : 50]
-Output [14]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160]
-Input [18]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, cd_marital_status#162, cd_demo_sk#163, cd_marital_status#164]
+(151) Project [codegen id : 51]
+Output [14]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161]
+Input [18]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_marital_status#163, cd_demo_sk#164, cd_marital_status#165]
-(151) ReusedExchange [Reuses operator id: 71]
-Output [1]: [p_promo_sk#165]
+(152) ReusedExchange [Reuses operator id: 71]
+Output [1]: [p_promo_sk#166]
-(152) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_promo_sk#126]
-Right keys [1]: [p_promo_sk#165]
+(153) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_promo_sk#127]
+Right keys [1]: [p_promo_sk#166]
Join condition: None
-(153) Project [codegen id : 50]
-Output [13]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160]
-Input [15]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, p_promo_sk#165]
+(154) Project [codegen id : 51]
+Output [13]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161]
+Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, p_promo_sk#166]
-(154) ReusedExchange [Reuses operator id: 77]
-Output [2]: [hd_demo_sk#166, hd_income_band_sk#167]
+(155) ReusedExchange [Reuses operator id: 77]
+Output [2]: [hd_demo_sk#167, hd_income_band_sk#168]
-(155) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_hdemo_sk#123]
-Right keys [1]: [hd_demo_sk#166]
+(156) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_hdemo_sk#124]
+Right keys [1]: [hd_demo_sk#167]
Join condition: None
-(156) Project [codegen id : 50]
-Output [13]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167]
-Input [15]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, hd_demo_sk#166, hd_income_band_sk#167]
+(157) Project [codegen id : 51]
+Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168]
+Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_demo_sk#167, hd_income_band_sk#168]
-(157) ReusedExchange [Reuses operator id: 77]
-Output [2]: [hd_demo_sk#168, hd_income_band_sk#169]
+(158) ReusedExchange [Reuses operator id: 77]
+Output [2]: [hd_demo_sk#169, hd_income_band_sk#170]
-(158) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [c_current_hdemo_sk#153]
-Right keys [1]: [hd_demo_sk#168]
+(159) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [c_current_hdemo_sk#154]
+Right keys [1]: [hd_demo_sk#169]
Join condition: None
-(159) Project [codegen id : 50]
-Output [13]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169]
-Input [15]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_demo_sk#168, hd_income_band_sk#169]
+(160) Project [codegen id : 51]
+Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170]
+Input [15]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_demo_sk#169, hd_income_band_sk#170]
-(160) ReusedExchange [Reuses operator id: 86]
-Output [5]: [ca_address_sk#170, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174]
+(161) ReusedExchange [Reuses operator id: 86]
+Output [5]: [ca_address_sk#171, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175]
-(161) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_addr_sk#124]
-Right keys [1]: [ca_address_sk#170]
+(162) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_addr_sk#125]
+Right keys [1]: [ca_address_sk#171]
Join condition: None
-(162) Project [codegen id : 50]
-Output [16]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174]
-Input [18]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_address_sk#170, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174]
+(163) Project [codegen id : 51]
+Output [16]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175]
+Input [18]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_address_sk#171, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175]
-(163) ReusedExchange [Reuses operator id: 86]
-Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179]
+(164) ReusedExchange [Reuses operator id: 86]
+Output [5]: [ca_address_sk#176, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180]
-(164) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [c_current_addr_sk#154]
-Right keys [1]: [ca_address_sk#175]
+(165) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [c_current_addr_sk#155]
+Right keys [1]: [ca_address_sk#176]
Join condition: None
-(165) Project [codegen id : 50]
-Output [19]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179]
-Input [21]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179]
+(166) Project [codegen id : 51]
+Output [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180]
+Input [21]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_address_sk#176, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180]
-(166) ReusedExchange [Reuses operator id: 95]
-Output [1]: [ib_income_band_sk#180]
+(167) ReusedExchange [Reuses operator id: 95]
+Output [1]: [ib_income_band_sk#181]
-(167) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [hd_income_band_sk#167]
-Right keys [1]: [ib_income_band_sk#180]
+(168) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [hd_income_band_sk#168]
+Right keys [1]: [ib_income_band_sk#181]
Join condition: None
-(168) Project [codegen id : 50]
-Output [18]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179]
-Input [20]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ib_income_band_sk#180]
+(169) Project [codegen id : 51]
+Output [18]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180]
+Input [20]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, ib_income_band_sk#181]
-(169) ReusedExchange [Reuses operator id: 95]
-Output [1]: [ib_income_band_sk#181]
+(170) ReusedExchange [Reuses operator id: 95]
+Output [1]: [ib_income_band_sk#182]
-(170) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [hd_income_band_sk#169]
-Right keys [1]: [ib_income_band_sk#181]
+(171) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [hd_income_band_sk#170]
+Right keys [1]: [ib_income_band_sk#182]
Join condition: None
-(171) Project [codegen id : 50]
-Output [17]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179]
-Input [19]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ib_income_band_sk#181]
+(172) Project [codegen id : 51]
+Output [17]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180]
+Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, ib_income_band_sk#182]
-(172) ReusedExchange [Reuses operator id: 105]
-Output [2]: [i_item_sk#182, i_product_name#183]
+(173) ReusedExchange [Reuses operator id: 105]
+Output [2]: [i_item_sk#183, i_product_name#184]
-(173) BroadcastHashJoin [codegen id : 50]
-Left keys [1]: [ss_item_sk#120]
-Right keys [1]: [i_item_sk#182]
+(174) BroadcastHashJoin [codegen id : 51]
+Left keys [1]: [ss_item_sk#121]
+Right keys [1]: [i_item_sk#183]
Join condition: None
-(174) Project [codegen id : 50]
-Output [18]: [ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, d_year#158, d_year#160, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, i_item_sk#182, i_product_name#183]
-Input [19]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, i_item_sk#182, i_product_name#183]
-
-(175) HashAggregate [codegen id : 50]
-Input [18]: [ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, d_year#158, d_year#160, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, i_item_sk#182, i_product_name#183]
-Keys [15]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160]
-Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#128)), partial_sum(UnscaledValue(ss_list_price#129)), partial_sum(UnscaledValue(ss_coupon_amt#130))]
-Aggregate Attributes [4]: [count#91, sum#184, sum#185, sum#186]
-Results [19]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160, count#95, sum#187, sum#188, sum#189]
-
-(176) HashAggregate [codegen id : 50]
-Input [19]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160, count#95, sum#187, sum#188, sum#189]
-Keys [15]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160]
-Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#128)), sum(UnscaledValue(ss_list_price#129)), sum(UnscaledValue(ss_coupon_amt#130))]
-Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#128))#100, sum(UnscaledValue(ss_list_price#129))#101, sum(UnscaledValue(ss_coupon_amt#130))#102]
-Results [8]: [i_item_sk#182 AS item_sk#190, s_store_name#149 AS store_name#191, s_zip#150 AS store_zip#192, d_year#147 AS syear#193, count(1)#99 AS cnt#194, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#128))#100,17,2) AS s1#195, MakeDecimal(sum(UnscaledValue(ss_list_price#129))#101,17,2) AS s2#196, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#130))#102,17,2) AS s3#197]
-
-(177) Sort [codegen id : 50]
-Input [8]: [item_sk#190, store_name#191, store_zip#192, syear#193, cnt#194, s1#195, s2#196, s3#197]
-Arguments: [item_sk#190 ASC NULLS FIRST, store_name#191 ASC NULLS FIRST, store_zip#192 ASC NULLS FIRST], false, 0
-
-(178) SortMergeJoin [codegen id : 51]
+(175) Project [codegen id : 51]
+Output [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, d_year#159, d_year#161, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184]
+Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184]
+
+(176) HashAggregate [codegen id : 51]
+Input [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, d_year#159, d_year#161, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184]
+Keys [15]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161]
+Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#129)), partial_sum(UnscaledValue(ss_list_price#130)), partial_sum(UnscaledValue(ss_coupon_amt#131))]
+Aggregate Attributes [4]: [count#91, sum#185, sum#186, sum#187]
+Results [19]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161, count#95, sum#188, sum#189, sum#190]
+
+(177) HashAggregate [codegen id : 51]
+Input [19]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161, count#95, sum#188, sum#189, sum#190]
+Keys [15]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161]
+Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#129)), sum(UnscaledValue(ss_list_price#130)), sum(UnscaledValue(ss_coupon_amt#131))]
+Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#129))#100, sum(UnscaledValue(ss_list_price#130))#101, sum(UnscaledValue(ss_coupon_amt#131))#102]
+Results [8]: [i_item_sk#183 AS item_sk#191, s_store_name#150 AS store_name#192, s_zip#151 AS store_zip#193, d_year#148 AS syear#194, count(1)#99 AS cnt#195, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#129))#100,17,2) AS s1#196, MakeDecimal(sum(UnscaledValue(ss_list_price#130))#101,17,2) AS s2#197, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#131))#102,17,2) AS s3#198]
+
+(178) Exchange
+Input [8]: [item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198]
+Arguments: hashpartitioning(item_sk#191, store_name#192, store_zip#193, 5), ENSURE_REQUIREMENTS, [id=#199]
+
+(179) Sort [codegen id : 52]
+Input [8]: [item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198]
+Arguments: [item_sk#191 ASC NULLS FIRST, store_name#192 ASC NULLS FIRST, store_zip#193 ASC NULLS FIRST], false, 0
+
+(180) SortMergeJoin [codegen id : 53]
Left keys [3]: [item_sk#104, store_name#105, store_zip#106]
-Right keys [3]: [item_sk#190, store_name#191, store_zip#192]
-Join condition: (cnt#194 <= cnt#116)
+Right keys [3]: [item_sk#191, store_name#192, store_zip#193]
+Join condition: (cnt#195 <= cnt#116)
-(179) Project [codegen id : 51]
-Output [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#195, s2#196, s3#197, syear#193, cnt#194]
-Input [25]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, item_sk#190, store_name#191, store_zip#192, syear#193, cnt#194, s1#195, s2#196, s3#197]
+(181) Project [codegen id : 53]
+Output [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195]
+Input [25]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198]
-(180) Exchange
-Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#195, s2#196, s3#197, syear#193, cnt#194]
-Arguments: rangepartitioning(product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#194 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#195 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#198]
+(182) Exchange
+Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195]
+Arguments: rangepartitioning(product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#195 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#196 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#200]
-(181) Sort [codegen id : 52]
-Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#195, s2#196, s3#197, syear#193, cnt#194]
-Arguments: [product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#194 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#195 ASC NULLS FIRST], true, 0
+(183) Sort [codegen id : 54]
+Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195]
+Arguments: [product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#195 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#196 ASC NULLS FIRST], true, 0
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13
-BroadcastExchange (185)
-+- * Filter (184)
- +- * ColumnarToRow (183)
- +- Scan parquet default.date_dim (182)
+BroadcastExchange (187)
++- * Filter (186)
+ +- * ColumnarToRow (185)
+ +- Scan parquet default.date_dim (184)
-(182) Scan parquet default.date_dim
+(184) Scan parquet default.date_dim
Output [2]: [d_date_sk#42, d_year#43]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
-(183) ColumnarToRow [codegen id : 1]
+(185) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#42, d_year#43]
-(184) Filter [codegen id : 1]
+(186) Filter [codegen id : 1]
Input [2]: [d_date_sk#42, d_year#43]
Condition : ((isnotnull(d_year#43) AND (d_year#43 = 1999)) AND isnotnull(d_date_sk#42))
-(185) BroadcastExchange
+(187) BroadcastExchange
Input [2]: [d_date_sk#42, d_year#43]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201]
-Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132
-BroadcastExchange (189)
-+- * Filter (188)
- +- * ColumnarToRow (187)
- +- Scan parquet default.date_dim (186)
+Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#132 IN dynamicpruning#133
+BroadcastExchange (191)
++- * Filter (190)
+ +- * ColumnarToRow (189)
+ +- Scan parquet default.date_dim (188)
-(186) Scan parquet default.date_dim
-Output [2]: [d_date_sk#146, d_year#147]
+(188) Scan parquet default.date_dim
+Output [2]: [d_date_sk#147, d_year#148]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>
-(187) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#146, d_year#147]
+(189) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#147, d_year#148]
-(188) Filter [codegen id : 1]
-Input [2]: [d_date_sk#146, d_year#147]
-Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146))
+(190) Filter [codegen id : 1]
+Input [2]: [d_date_sk#147, d_year#148]
+Condition : ((isnotnull(d_year#148) AND (d_year#148 = 2000)) AND isnotnull(d_date_sk#147))
-(189) BroadcastExchange
-Input [2]: [d_date_sk#146, d_year#147]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200]
+(191) BroadcastExchange
+Input [2]: [d_date_sk#147, d_year#148]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202]
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 716aaa2..6917f8f 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
@@ -1,283 +1,289 @@
-WholeStageCodegen (52)
+WholeStageCodegen (54)
Sort [product_name,store_name,cnt,s1,s1]
InputAdapter
Exchange [product_name,store_name,cnt,s1,s1] #1
- WholeStageCodegen (51)
+ WholeStageCodegen (53)
Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt]
SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt]
InputAdapter
- WholeStageCodegen (25)
+ WholeStageCodegen (26)
Sort [item_sk,store_name,store_zip]
- HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum]
- HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum]
- Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name]
- BroadcastHashJoin [ss_item_sk,i_item_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [ss_addr_sk,ca_address_sk]
- Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk]
- BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
- Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk]
- BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
- Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
- BroadcastHashJoin [ss_promo_sk,p_promo_sk]
- Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
- BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status]
- Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status]
- BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk]
- Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
- BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk]
- Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year]
- BroadcastHashJoin [c_first_sales_date_sk,d_date_sk]
- Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
- BroadcastHashJoin [ss_customer_sk,c_customer_sk]
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- SortMergeJoin [ss_item_sk,cs_item_sk]
- InputAdapter
- WholeStageCodegen (3)
- Sort [ss_item_sk]
+ InputAdapter
+ Exchange [item_sk,store_name,store_zip] #2
+ WholeStageCodegen (25)
+ HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum, [...]
+ HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum]
+ Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk]
+ BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk]
+ BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
+ BroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
+ BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status]
+ Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status]
+ BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk]
+ Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
+ BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk]
+ Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year]
+ BroadcastHashJoin [c_first_sales_date_sk,d_date_sk]
+ Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ SortMergeJoin [ss_item_sk,cs_item_sk]
InputAdapter
- Exchange [ss_item_sk] #2
- WholeStageCodegen (2)
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- InputAdapter
- BroadcastExchange #3
- WholeStageCodegen (1)
- Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #4
- WholeStageCodegen (1)
- Filter [d_year,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year]
- Project [sr_item_sk,sr_ticket_number]
- Filter [sr_item_sk,sr_ticket_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
- InputAdapter
- WholeStageCodegen (9)
- Sort [cs_item_sk]
- Project [cs_item_sk]
- Filter [sale,refund]
- 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, [...]
+ WholeStageCodegen (3)
+ Sort [ss_item_sk]
InputAdapter
- Exchange [cs_item_sk] #5
- 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
- WholeStageCodegen (5)
- Sort [cs_item_sk,cs_order_number]
+ Exchange [ss_item_sk] #3
+ WholeStageCodegen (2)
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (1)
+ Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ Filter [d_year,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year]
+ Project [sr_item_sk,sr_ticket_number]
+ Filter [sr_item_sk,sr_ticket_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
+ InputAdapter
+ WholeStageCodegen (9)
+ Sort [cs_item_sk]
+ Project [cs_item_sk]
+ Filter [sale,refund]
+ 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)) [...]
+ 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]
+ 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
- Exchange [cs_item_sk,cs_order_number] #6
- WholeStageCodegen (4)
- Project [cs_item_sk,cs_order_number,cs_ext_list_price]
- Filter [cs_item_sk,cs_order_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
- InputAdapter
- WholeStageCodegen (7)
- Sort [cr_item_sk,cr_order_number]
+ WholeStageCodegen (5)
+ Sort [cs_item_sk,cs_order_number]
+ InputAdapter
+ Exchange [cs_item_sk,cs_order_number] #7
+ WholeStageCodegen (4)
+ Project [cs_item_sk,cs_order_number,cs_ext_list_price]
+ Filter [cs_item_sk,cs_order_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
- Exchange [cr_item_sk,cr_order_number] #7
- WholeStageCodegen (6)
- Project [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- Filter [cr_item_sk,cr_order_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
- InputAdapter
- ReusedExchange [d_date_sk,d_year] #4
- InputAdapter
- BroadcastExchange #8
- WholeStageCodegen (11)
- Filter [s_store_sk,s_store_name,s_zip]
- ColumnarToRow
+ WholeStageCodegen (7)
+ Sort [cr_item_sk,cr_order_number]
+ InputAdapter
+ Exchange [cr_item_sk,cr_order_number] #8
+ WholeStageCodegen (6)
+ Project [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
+ Filter [cr_item_sk,cr_order_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
- Scan parquet default.store [s_store_sk,s_store_name,s_zip]
- InputAdapter
- BroadcastExchange #9
- WholeStageCodegen (12)
- Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- ColumnarToRow
+ ReusedExchange [d_date_sk,d_year] #5
InputAdapter
- Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
- InputAdapter
- BroadcastExchange #10
- WholeStageCodegen (13)
- Filter [d_date_sk]
- ColumnarToRow
+ BroadcastExchange #9
+ WholeStageCodegen (11)
+ Filter [s_store_sk,s_store_name,s_zip]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store [s_store_sk,s_store_name,s_zip]
InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year]
- InputAdapter
- ReusedExchange [d_date_sk,d_year] #10
- InputAdapter
- BroadcastExchange #11
- WholeStageCodegen (15)
- Filter [cd_demo_sk,cd_marital_status]
- ColumnarToRow
+ BroadcastExchange #10
+ WholeStageCodegen (12)
+ Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
+ InputAdapter
+ BroadcastExchange #11
+ WholeStageCodegen (13)
+ Filter [d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year]
InputAdapter
- Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status]
- InputAdapter
- ReusedExchange [cd_demo_sk,cd_marital_status] #11
- InputAdapter
- BroadcastExchange #12
- WholeStageCodegen (17)
- Filter [p_promo_sk]
- ColumnarToRow
+ ReusedExchange [d_date_sk,d_year] #11
+ InputAdapter
+ BroadcastExchange #12
+ WholeStageCodegen (15)
+ Filter [cd_demo_sk,cd_marital_status]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status]
InputAdapter
- Scan parquet default.promotion [p_promo_sk]
- InputAdapter
- BroadcastExchange #13
- WholeStageCodegen (18)
- Filter [hd_demo_sk,hd_income_band_sk]
- ColumnarToRow
+ ReusedExchange [cd_demo_sk,cd_marital_status] #12
InputAdapter
- Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk]
- InputAdapter
- ReusedExchange [hd_demo_sk,hd_income_band_sk] #13
- InputAdapter
- BroadcastExchange #14
- WholeStageCodegen (20)
- Filter [ca_address_sk]
- ColumnarToRow
+ BroadcastExchange #13
+ WholeStageCodegen (17)
+ Filter [p_promo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.promotion [p_promo_sk]
+ InputAdapter
+ BroadcastExchange #14
+ WholeStageCodegen (18)
+ Filter [hd_demo_sk,hd_income_band_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk]
InputAdapter
- Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
- InputAdapter
- ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14
- InputAdapter
- BroadcastExchange #15
- WholeStageCodegen (22)
- Filter [ib_income_band_sk]
+ ReusedExchange [hd_demo_sk,hd_income_band_sk] #14
+ InputAdapter
+ BroadcastExchange #15
+ WholeStageCodegen (20)
+ Filter [ca_address_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
+ InputAdapter
+ ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
+ InputAdapter
+ BroadcastExchange #16
+ WholeStageCodegen (22)
+ Filter [ib_income_band_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.income_band [ib_income_band_sk]
+ InputAdapter
+ ReusedExchange [ib_income_band_sk] #16
+ InputAdapter
+ BroadcastExchange #17
+ WholeStageCodegen (24)
+ Project [i_item_sk,i_product_name]
+ Filter [i_current_price,i_color,i_item_sk]
ColumnarToRow
InputAdapter
- Scan parquet default.income_band [ib_income_band_sk]
- InputAdapter
- ReusedExchange [ib_income_band_sk] #15
- InputAdapter
- BroadcastExchange #16
- WholeStageCodegen (24)
- Project [i_item_sk,i_product_name]
- Filter [i_current_price,i_color,i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name]
+ Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
- WholeStageCodegen (50)
+ WholeStageCodegen (52)
Sort [item_sk,store_name,store_zip]
- HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum]
- HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum]
- Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name]
- BroadcastHashJoin [ss_item_sk,i_item_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
- Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
- BroadcastHashJoin [ss_addr_sk,ca_address_sk]
- Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk]
- BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
- Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk]
- BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
- Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
- BroadcastHashJoin [ss_promo_sk,p_promo_sk]
- Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
- BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status]
- Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status]
- BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk]
- Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
- BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk]
- Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year]
- BroadcastHashJoin [c_first_sales_date_sk,d_date_sk]
- Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
- BroadcastHashJoin [ss_customer_sk,c_customer_sk]
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip]
- BroadcastHashJoin [ss_store_sk,s_store_sk]
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year]
- BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- SortMergeJoin [ss_item_sk,cs_item_sk]
- InputAdapter
- WholeStageCodegen (28)
- Sort [ss_item_sk]
+ InputAdapter
+ Exchange [item_sk,store_name,store_zip] #18
+ WholeStageCodegen (51)
+ HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum]
+ HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum]
+ Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
+ BroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk]
+ BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk]
+ BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
+ BroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
+ BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status]
+ Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status]
+ BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk]
+ Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year]
+ BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk]
+ Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year]
+ BroadcastHashJoin [c_first_sales_date_sk,d_date_sk]
+ Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year]
+ BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ SortMergeJoin [ss_item_sk,cs_item_sk]
InputAdapter
- Exchange [ss_item_sk] #17
- WholeStageCodegen (27)
- Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- InputAdapter
- BroadcastExchange #18
- WholeStageCodegen (26)
- Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #2
- BroadcastExchange #19
- WholeStageCodegen (1)
- Filter [d_year,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_year]
- Project [sr_item_sk,sr_ticket_number]
- Filter [sr_item_sk,sr_ticket_number]
- ColumnarToRow
- InputAdapter
- Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
- InputAdapter
- WholeStageCodegen (34)
- Sort [cs_item_sk]
- Project [cs_item_sk]
- Filter [sale,refund]
- 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, [...]
+ WholeStageCodegen (29)
+ Sort [ss_item_sk]
InputAdapter
- ReusedExchange [cs_item_sk,sum,sum,isEmpty] #5
- InputAdapter
- ReusedExchange [d_date_sk,d_year] #19
- InputAdapter
- ReusedExchange [s_store_sk,s_store_name,s_zip] #8
- InputAdapter
- ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #9
- InputAdapter
- ReusedExchange [d_date_sk,d_year] #10
- InputAdapter
- ReusedExchange [d_date_sk,d_year] #10
- InputAdapter
- ReusedExchange [cd_demo_sk,cd_marital_status] #11
- InputAdapter
- ReusedExchange [cd_demo_sk,cd_marital_status] #11
- InputAdapter
- ReusedExchange [p_promo_sk] #12
- InputAdapter
- ReusedExchange [hd_demo_sk,hd_income_band_sk] #13
- InputAdapter
- ReusedExchange [hd_demo_sk,hd_income_band_sk] #13
- InputAdapter
- ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14
- InputAdapter
- ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14
- InputAdapter
- ReusedExchange [ib_income_band_sk] #15
- InputAdapter
- ReusedExchange [ib_income_band_sk] #15
- InputAdapter
- ReusedExchange [i_item_sk,i_product_name] #16
+ Exchange [ss_item_sk] #19
+ WholeStageCodegen (28)
+ Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
+ InputAdapter
+ BroadcastExchange #20
+ WholeStageCodegen (27)
+ Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #21
+ WholeStageCodegen (1)
+ Filter [d_year,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_year]
+ Project [sr_item_sk,sr_ticket_number]
+ Filter [sr_item_sk,sr_ticket_number]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
+ InputAdapter
+ WholeStageCodegen (35)
+ Sort [cs_item_sk]
+ Project [cs_item_sk]
+ Filter [sale,refund]
+ 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)) [...]
+ InputAdapter
+ ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6
+ InputAdapter
+ ReusedExchange [d_date_sk,d_year] #21
+ InputAdapter
+ ReusedExchange [s_store_sk,s_store_name,s_zip] #9
+ InputAdapter
+ ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10
+ InputAdapter
+ ReusedExchange [d_date_sk,d_year] #11
+ InputAdapter
+ ReusedExchange [d_date_sk,d_year] #11
+ InputAdapter
+ ReusedExchange [cd_demo_sk,cd_marital_status] #12
+ InputAdapter
+ ReusedExchange [cd_demo_sk,cd_marital_status] #12
+ InputAdapter
+ ReusedExchange [p_promo_sk] #13
+ InputAdapter
+ ReusedExchange [hd_demo_sk,hd_income_band_sk] #14
+ InputAdapter
+ ReusedExchange [hd_demo_sk,hd_income_band_sk] #14
+ InputAdapter
+ ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
+ InputAdapter
+ ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
+ InputAdapter
+ ReusedExchange [ib_income_band_sk] #16
+ InputAdapter
+ ReusedExchange [ib_income_band_sk] #16
+ InputAdapter
+ ReusedExchange [i_item_sk,i_product_name] #17
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt
index 42f7488..e5e42f2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt
@@ -1,72 +1,74 @@
== Physical Plan ==
-TakeOrderedAndProject (68)
-+- * HashAggregate (67)
- +- Exchange (66)
- +- * HashAggregate (65)
- +- * Project (64)
- +- * SortMergeJoin LeftOuter (63)
- :- * Sort (56)
- : +- * Project (55)
- : +- * BroadcastHashJoin LeftOuter BuildRight (54)
- : :- * Project (49)
- : : +- * SortMergeJoin Inner (48)
- : : :- * Sort (36)
- : : : +- * Project (35)
- : : : +- * BroadcastHashJoin Inner BuildRight (34)
- : : : :- * Project (32)
- : : : : +- * SortMergeJoin Inner (31)
- : : : : :- * Sort (25)
- : : : : : +- Exchange (24)
- : : : : : +- * Project (23)
- : : : : : +- * BroadcastHashJoin Inner BuildRight (22)
- : : : : : :- * Project (17)
- : : : : : : +- * BroadcastHashJoin Inner BuildRight (16)
- : : : : : : :- * Project (10)
- : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9)
- : : : : : : : :- * Filter (3)
- : : : : : : : : +- * ColumnarToRow (2)
- : : : : : : : : +- Scan parquet default.catalog_sales (1)
- : : : : : : : +- BroadcastExchange (8)
- : : : : : : : +- * Project (7)
- : : : : : : : +- * Filter (6)
- : : : : : : : +- * ColumnarToRow (5)
- : : : : : : : +- Scan parquet default.household_demographics (4)
- : : : : : : +- BroadcastExchange (15)
- : : : : : : +- * Project (14)
- : : : : : : +- * Filter (13)
- : : : : : : +- * ColumnarToRow (12)
- : : : : : : +- Scan parquet default.customer_demographics (11)
- : : : : : +- BroadcastExchange (21)
- : : : : : +- * Filter (20)
- : : : : : +- * ColumnarToRow (19)
- : : : : : +- Scan parquet default.date_dim (18)
- : : : : +- * Sort (30)
- : : : : +- Exchange (29)
- : : : : +- * Filter (28)
- : : : : +- * ColumnarToRow (27)
- : : : : +- Scan parquet default.item (26)
- : : : +- ReusedExchange (33)
- : : +- * Sort (47)
- : : +- Exchange (46)
- : : +- * Project (45)
- : : +- * BroadcastHashJoin Inner BuildRight (44)
- : : :- * Filter (39)
- : : : +- * ColumnarToRow (38)
- : : : +- Scan parquet default.inventory (37)
- : : +- BroadcastExchange (43)
- : : +- * Filter (42)
- : : +- * ColumnarToRow (41)
- : : +- Scan parquet default.warehouse (40)
- : +- BroadcastExchange (53)
- : +- * Filter (52)
- : +- * ColumnarToRow (51)
- : +- Scan parquet default.promotion (50)
- +- * Sort (62)
- +- Exchange (61)
- +- * Project (60)
- +- * Filter (59)
- +- * ColumnarToRow (58)
- +- Scan parquet default.catalog_returns (57)
+TakeOrderedAndProject (70)
++- * HashAggregate (69)
+ +- Exchange (68)
+ +- * HashAggregate (67)
+ +- * Project (66)
+ +- * SortMergeJoin LeftOuter (65)
+ :- * Sort (58)
+ : +- Exchange (57)
+ : +- * Project (56)
+ : +- * BroadcastHashJoin LeftOuter BuildRight (55)
+ : :- * Project (50)
+ : : +- * SortMergeJoin Inner (49)
+ : : :- * Sort (37)
+ : : : +- Exchange (36)
+ : : : +- * Project (35)
+ : : : +- * BroadcastHashJoin Inner BuildRight (34)
+ : : : :- * Project (32)
+ : : : : +- * SortMergeJoin Inner (31)
+ : : : : :- * Sort (25)
+ : : : : : +- Exchange (24)
+ : : : : : +- * Project (23)
+ : : : : : +- * BroadcastHashJoin Inner BuildRight (22)
+ : : : : : :- * Project (17)
+ : : : : : : +- * BroadcastHashJoin Inner BuildRight (16)
+ : : : : : : :- * Project (10)
+ : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9)
+ : : : : : : : :- * Filter (3)
+ : : : : : : : : +- * ColumnarToRow (2)
+ : : : : : : : : +- Scan parquet default.catalog_sales (1)
+ : : : : : : : +- BroadcastExchange (8)
+ : : : : : : : +- * Project (7)
+ : : : : : : : +- * Filter (6)
+ : : : : : : : +- * ColumnarToRow (5)
+ : : : : : : : +- Scan parquet default.household_demographics (4)
+ : : : : : : +- BroadcastExchange (15)
+ : : : : : : +- * Project (14)
+ : : : : : : +- * Filter (13)
+ : : : : : : +- * ColumnarToRow (12)
+ : : : : : : +- Scan parquet default.customer_demographics (11)
+ : : : : : +- BroadcastExchange (21)
+ : : : : : +- * Filter (20)
+ : : : : : +- * ColumnarToRow (19)
+ : : : : : +- Scan parquet default.date_dim (18)
+ : : : : +- * Sort (30)
+ : : : : +- Exchange (29)
+ : : : : +- * Filter (28)
+ : : : : +- * ColumnarToRow (27)
+ : : : : +- Scan parquet default.item (26)
+ : : : +- ReusedExchange (33)
+ : : +- * Sort (48)
+ : : +- Exchange (47)
+ : : +- * Project (46)
+ : : +- * BroadcastHashJoin Inner BuildRight (45)
+ : : :- * Filter (40)
+ : : : +- * ColumnarToRow (39)
+ : : : +- Scan parquet default.inventory (38)
+ : : +- BroadcastExchange (44)
+ : : +- * Filter (43)
+ : : +- * ColumnarToRow (42)
+ : : +- Scan parquet default.warehouse (41)
+ : +- BroadcastExchange (54)
+ : +- * Filter (53)
+ : +- * ColumnarToRow (52)
+ : +- Scan parquet default.promotion (51)
+ +- * Sort (64)
+ +- Exchange (63)
+ +- * Project (62)
+ +- * Filter (61)
+ +- * ColumnarToRow (60)
+ +- Scan parquet default.catalog_returns (59)
(1) Scan parquet default.catalog_sales
@@ -212,7 +214,7 @@ Join condition: None
Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21]
Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21]
-(33) ReusedExchange [Reuses operator id: 79]
+(33) ReusedExchange [Reuses operator id: 81]
Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
(34) BroadcastHashJoin [codegen id : 10]
@@ -224,220 +226,228 @@ Join condition: (d_date#17 > date_add(d_date#24, 5))
Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26]
Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
-(36) Sort [codegen id : 10]
+(36) Exchange
+Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26]
+Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#26, 5), ENSURE_REQUIREMENTS, [id=#27]
+
+(37) Sort [codegen id : 11]
Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26]
Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#26 ASC NULLS FIRST], false, 0
-(37) Scan parquet default.inventory
-Output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30]
+(38) Scan parquet default.inventory
+Output [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31]
Batched: true
Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(true)]
+PartitionFilters: [isnotnull(inv_date_sk#31), dynamicpruningexpression(true)]
PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)]
ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int>
-(38) ColumnarToRow [codegen id : 12]
-Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30]
+(39) ColumnarToRow [codegen id : 13]
+Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31]
-(39) Filter [codegen id : 12]
-Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30]
-Condition : ((isnotnull(inv_quantity_on_hand#29) AND isnotnull(inv_item_sk#27)) AND isnotnull(inv_warehouse_sk#28))
+(40) Filter [codegen id : 13]
+Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31]
+Condition : ((isnotnull(inv_quantity_on_hand#30) AND isnotnull(inv_item_sk#28)) AND isnotnull(inv_warehouse_sk#29))
-(40) Scan parquet default.warehouse
-Output [2]: [w_warehouse_sk#31, w_warehouse_name#32]
+(41) Scan parquet default.warehouse
+Output [2]: [w_warehouse_sk#32, w_warehouse_name#33]
Batched: true
Location [not included in comparison]/{warehouse_dir}/warehouse]
PushedFilters: [IsNotNull(w_warehouse_sk)]
ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string>
-(41) ColumnarToRow [codegen id : 11]
-Input [2]: [w_warehouse_sk#31, w_warehouse_name#32]
+(42) ColumnarToRow [codegen id : 12]
+Input [2]: [w_warehouse_sk#32, w_warehouse_name#33]
-(42) Filter [codegen id : 11]
-Input [2]: [w_warehouse_sk#31, w_warehouse_name#32]
-Condition : isnotnull(w_warehouse_sk#31)
+(43) Filter [codegen id : 12]
+Input [2]: [w_warehouse_sk#32, w_warehouse_name#33]
+Condition : isnotnull(w_warehouse_sk#32)
-(43) BroadcastExchange
-Input [2]: [w_warehouse_sk#31, w_warehouse_name#32]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33]
+(44) BroadcastExchange
+Input [2]: [w_warehouse_sk#32, w_warehouse_name#33]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34]
-(44) BroadcastHashJoin [codegen id : 12]
-Left keys [1]: [inv_warehouse_sk#28]
-Right keys [1]: [w_warehouse_sk#31]
+(45) BroadcastHashJoin [codegen id : 13]
+Left keys [1]: [inv_warehouse_sk#29]
+Right keys [1]: [w_warehouse_sk#32]
Join condition: None
-(45) Project [codegen id : 12]
-Output [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
-Input [6]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_sk#31, w_warehouse_name#32]
+(46) Project [codegen id : 13]
+Output [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
+Input [6]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_sk#32, w_warehouse_name#33]
-(46) Exchange
-Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
-Arguments: hashpartitioning(inv_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#34]
+(47) Exchange
+Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
+Arguments: hashpartitioning(inv_item_sk#28, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#35]
-(47) Sort [codegen id : 13]
-Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
-Arguments: [inv_item_sk#27 ASC NULLS FIRST, inv_date_sk#30 ASC NULLS FIRST], false, 0
+(48) Sort [codegen id : 14]
+Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
+Arguments: [inv_item_sk#28 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0
-(48) SortMergeJoin [codegen id : 15]
+(49) SortMergeJoin [codegen id : 16]
Left keys [2]: [cs_item_sk#4, d_date_sk#26]
-Right keys [2]: [inv_item_sk#27, inv_date_sk#30]
-Join condition: (inv_quantity_on_hand#29 < cs_quantity#7)
+Right keys [2]: [inv_item_sk#28, inv_date_sk#31]
+Join condition: (inv_quantity_on_hand#30 < cs_quantity#7)
-(49) Project [codegen id : 15]
-Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32]
+(50) Project [codegen id : 16]
+Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33]
-(50) Scan parquet default.promotion
-Output [1]: [p_promo_sk#35]
+(51) Scan parquet default.promotion
+Output [1]: [p_promo_sk#36]
Batched: true
Location [not included in comparison]/{warehouse_dir}/promotion]
PushedFilters: [IsNotNull(p_promo_sk)]
ReadSchema: struct<p_promo_sk:int>
-(51) ColumnarToRow [codegen id : 14]
-Input [1]: [p_promo_sk#35]
+(52) ColumnarToRow [codegen id : 15]
+Input [1]: [p_promo_sk#36]
-(52) Filter [codegen id : 14]
-Input [1]: [p_promo_sk#35]
-Condition : isnotnull(p_promo_sk#35)
+(53) Filter [codegen id : 15]
+Input [1]: [p_promo_sk#36]
+Condition : isnotnull(p_promo_sk#36)
-(53) BroadcastExchange
-Input [1]: [p_promo_sk#35]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36]
+(54) BroadcastExchange
+Input [1]: [p_promo_sk#36]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37]
-(54) BroadcastHashJoin [codegen id : 15]
+(55) BroadcastHashJoin [codegen id : 16]
Left keys [1]: [cs_promo_sk#5]
-Right keys [1]: [p_promo_sk#35]
+Right keys [1]: [p_promo_sk#36]
Join condition: None
-(55) Project [codegen id : 15]
-Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, p_promo_sk#35]
+(56) Project [codegen id : 16]
+Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, p_promo_sk#36]
+
+(57) Exchange
+Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38]
-(56) Sort [codegen id : 15]
-Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
+(58) Sort [codegen id : 17]
+Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0
-(57) Scan parquet default.catalog_returns
-Output [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
+(59) Scan parquet default.catalog_returns
+Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
Batched: true
Location [not included in comparison]/{warehouse_dir}/catalog_returns]
PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)]
ReadSchema: struct<cr_item_sk:int,cr_order_number:int>
-(58) ColumnarToRow [codegen id : 16]
-Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
+(60) ColumnarToRow [codegen id : 18]
+Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
-(59) Filter [codegen id : 16]
-Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
-Condition : (isnotnull(cr_item_sk#37) AND isnotnull(cr_order_number#38))
+(61) Filter [codegen id : 18]
+Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
+Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40))
-(60) Project [codegen id : 16]
-Output [2]: [cr_item_sk#37, cr_order_number#38]
-Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39]
+(62) Project [codegen id : 18]
+Output [2]: [cr_item_sk#39, cr_order_number#40]
+Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41]
-(61) Exchange
-Input [2]: [cr_item_sk#37, cr_order_number#38]
-Arguments: hashpartitioning(cr_item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#40]
+(63) Exchange
+Input [2]: [cr_item_sk#39, cr_order_number#40]
+Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42]
-(62) Sort [codegen id : 17]
-Input [2]: [cr_item_sk#37, cr_order_number#38]
-Arguments: [cr_item_sk#37 ASC NULLS FIRST, cr_order_number#38 ASC NULLS FIRST], false, 0
+(64) Sort [codegen id : 19]
+Input [2]: [cr_item_sk#39, cr_order_number#40]
+Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0
-(63) SortMergeJoin [codegen id : 18]
+(65) SortMergeJoin [codegen id : 20]
Left keys [2]: [cs_item_sk#4, cs_order_number#6]
-Right keys [2]: [cr_item_sk#37, cr_order_number#38]
+Right keys [2]: [cr_item_sk#39, cr_order_number#40]
Join condition: None
-(64) Project [codegen id : 18]
-Output [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, cr_item_sk#37, cr_order_number#38]
+(66) Project [codegen id : 20]
+Output [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, cr_item_sk#39, cr_order_number#40]
-(65) HashAggregate [codegen id : 18]
-Input [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25]
-Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25]
+(67) HashAggregate [codegen id : 20]
+Input [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25]
+Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25]
Functions [1]: [partial_count(1)]
-Aggregate Attributes [1]: [count#41]
-Results [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42]
+Aggregate Attributes [1]: [count#43]
+Results [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44]
-(66) Exchange
-Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42]
-Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#32, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#43]
+(68) Exchange
+Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44]
+Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#33, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#45]
-(67) HashAggregate [codegen id : 19]
-Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42]
-Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25]
+(69) HashAggregate [codegen id : 21]
+Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44]
+Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25]
Functions [1]: [count(1)]
-Aggregate Attributes [1]: [count(1)#44]
-Results [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count(1)#44 AS no_promo#45, count(1)#44 AS promo#46, count(1)#44 AS total_cnt#47]
+Aggregate Attributes [1]: [count(1)#46]
+Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49]
-(68) TakeOrderedAndProject
-Input [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47]
-Arguments: 100, [total_cnt#47 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#32 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47]
+(70) TakeOrderedAndProject
+Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
+Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
===== Subqueries =====
Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9
-BroadcastExchange (79)
-+- * Project (78)
- +- * BroadcastHashJoin Inner BuildLeft (77)
- :- BroadcastExchange (73)
- : +- * Project (72)
- : +- * Filter (71)
- : +- * ColumnarToRow (70)
- : +- Scan parquet default.date_dim (69)
- +- * Filter (76)
- +- * ColumnarToRow (75)
- +- Scan parquet default.date_dim (74)
-
-
-(69) Scan parquet default.date_dim
-Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
+BroadcastExchange (81)
++- * Project (80)
+ +- * BroadcastHashJoin Inner BuildLeft (79)
+ :- BroadcastExchange (75)
+ : +- * Project (74)
+ : +- * Filter (73)
+ : +- * ColumnarToRow (72)
+ : +- Scan parquet default.date_dim (71)
+ +- * Filter (78)
+ +- * ColumnarToRow (77)
+ +- Scan parquet default.date_dim (76)
+
+
+(71) Scan parquet default.date_dim
+Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)]
ReadSchema: struct<d_date_sk:int,d_date:date,d_week_seq:int,d_year:int>
-(70) ColumnarToRow [codegen id : 1]
-Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
+(72) ColumnarToRow [codegen id : 1]
+Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
-(71) Filter [codegen id : 1]
-Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
-Condition : ((((isnotnull(d_year#48) AND (d_year#48 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24))
+(73) Filter [codegen id : 1]
+Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
+Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24))
-(72) Project [codegen id : 1]
+(74) Project [codegen id : 1]
Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25]
-Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48]
+Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50]
-(73) BroadcastExchange
+(75) BroadcastExchange
Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25]
-Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49]
+Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51]
-(74) Scan parquet default.date_dim
-Output [2]: [d_date_sk#26, d_week_seq#50]
+(76) Scan parquet default.date_dim
+Output [2]: [d_date_sk#26, d_week_seq#52]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_week_seq:int>
-(75) ColumnarToRow
-Input [2]: [d_date_sk#26, d_week_seq#50]
+(77) ColumnarToRow
+Input [2]: [d_date_sk#26, d_week_seq#52]
-(76) Filter
-Input [2]: [d_date_sk#26, d_week_seq#50]
-Condition : (isnotnull(d_week_seq#50) AND isnotnull(d_date_sk#26))
+(78) Filter
+Input [2]: [d_date_sk#26, d_week_seq#52]
+Condition : (isnotnull(d_week_seq#52) AND isnotnull(d_date_sk#26))
-(77) BroadcastHashJoin [codegen id : 2]
+(79) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [d_week_seq#25]
-Right keys [1]: [d_week_seq#50]
+Right keys [1]: [d_week_seq#52]
Join condition: None
-(78) Project [codegen id : 2]
+(80) Project [codegen id : 2]
Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
-Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50]
+Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#52]
-(79) BroadcastExchange
+(81) BroadcastExchange
Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt
index d84393b..e838025 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt
@@ -1,126 +1,132 @@
TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo]
- WholeStageCodegen (19)
+ WholeStageCodegen (21)
HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count]
InputAdapter
Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1
- WholeStageCodegen (18)
+ WholeStageCodegen (20)
HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count]
Project [w_warehouse_name,i_item_desc,d_week_seq]
SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number]
InputAdapter
- WholeStageCodegen (15)
+ WholeStageCodegen (17)
Sort [cs_item_sk,cs_order_number]
- Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
- BroadcastHashJoin [cs_promo_sk,p_promo_sk]
- Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
- SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity]
- InputAdapter
- WholeStageCodegen (10)
- Sort [cs_item_sk,d_date_sk]
- Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk]
- BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date]
- Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc]
- SortMergeJoin [cs_item_sk,i_item_sk]
- InputAdapter
- WholeStageCodegen (5)
- Sort [cs_item_sk]
- InputAdapter
- Exchange [cs_item_sk] #2
- WholeStageCodegen (4)
- Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date]
- BroadcastHashJoin [cs_ship_date_sk,d_date_sk]
- Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
- BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
- Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
- BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
- Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
- SubqueryBroadcast [d_date_sk] #1
- BroadcastExchange #3
- WholeStageCodegen (2)
- Project [d_date_sk,d_date,d_week_seq,d_date_sk]
- BroadcastHashJoin [d_week_seq,d_week_seq]
- InputAdapter
- BroadcastExchange #4
- WholeStageCodegen (1)
- Project [d_date_sk,d_date,d_week_seq]
- Filter [d_year,d_date_sk,d_week_seq,d_date]
- ColumnarToRow
+ InputAdapter
+ Exchange [cs_item_sk,cs_order_number] #2
+ WholeStageCodegen (16)
+ Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
+ BroadcastHashJoin [cs_promo_sk,p_promo_sk]
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq]
+ SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity]
+ InputAdapter
+ WholeStageCodegen (11)
+ Sort [cs_item_sk,d_date_sk]
+ InputAdapter
+ Exchange [cs_item_sk,d_date_sk] #3
+ WholeStageCodegen (10)
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date]
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc]
+ SortMergeJoin [cs_item_sk,i_item_sk]
+ InputAdapter
+ WholeStageCodegen (5)
+ Sort [cs_item_sk]
+ InputAdapter
+ Exchange [cs_item_sk] #4
+ WholeStageCodegen (4)
+ Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date]
+ BroadcastHashJoin [cs_ship_date_sk,d_date_sk]
+ Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
+ BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
+ BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #5
+ WholeStageCodegen (2)
+ Project [d_date_sk,d_date,d_week_seq,d_date_sk]
+ BroadcastHashJoin [d_week_seq,d_week_seq]
InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- Filter [d_week_seq,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_week_seq]
- InputAdapter
- BroadcastExchange #5
- WholeStageCodegen (1)
- Project [hd_demo_sk]
- Filter [hd_buy_potential,hd_demo_sk]
- ColumnarToRow
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ Project [d_date_sk,d_date,d_week_seq]
+ Filter [d_year,d_date_sk,d_week_seq,d_date]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
+ Filter [d_week_seq,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_week_seq]
InputAdapter
- Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential]
- InputAdapter
- BroadcastExchange #6
- WholeStageCodegen (2)
- Project [cd_demo_sk]
- Filter [cd_marital_status,cd_demo_sk]
- ColumnarToRow
+ BroadcastExchange #7
+ WholeStageCodegen (1)
+ Project [hd_demo_sk]
+ Filter [hd_buy_potential,hd_demo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential]
InputAdapter
- Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status]
- InputAdapter
- BroadcastExchange #7
- WholeStageCodegen (3)
- Filter [d_date,d_date_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.date_dim [d_date_sk,d_date]
- InputAdapter
- WholeStageCodegen (7)
- Sort [i_item_sk]
- InputAdapter
- Exchange [i_item_sk] #8
- WholeStageCodegen (6)
- Filter [i_item_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.item [i_item_sk,i_item_desc]
- InputAdapter
- ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #3
- InputAdapter
- WholeStageCodegen (13)
- Sort [inv_item_sk,inv_date_sk]
+ BroadcastExchange #8
+ WholeStageCodegen (2)
+ Project [cd_demo_sk]
+ Filter [cd_marital_status,cd_demo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status]
+ InputAdapter
+ BroadcastExchange #9
+ WholeStageCodegen (3)
+ Filter [d_date,d_date_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.date_dim [d_date_sk,d_date]
+ InputAdapter
+ WholeStageCodegen (7)
+ Sort [i_item_sk]
+ InputAdapter
+ Exchange [i_item_sk] #10
+ WholeStageCodegen (6)
+ Filter [i_item_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.item [i_item_sk,i_item_desc]
+ InputAdapter
+ ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5
InputAdapter
- Exchange [inv_item_sk] #9
- WholeStageCodegen (12)
- Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name]
- BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
- Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- InputAdapter
- BroadcastExchange #10
- WholeStageCodegen (11)
- Filter [w_warehouse_sk]
+ WholeStageCodegen (14)
+ Sort [inv_item_sk,inv_date_sk]
+ InputAdapter
+ Exchange [inv_item_sk,inv_date_sk] #11
+ WholeStageCodegen (13)
+ Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name]
+ BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
ColumnarToRow
InputAdapter
- Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name]
- InputAdapter
- BroadcastExchange #11
- WholeStageCodegen (14)
- Filter [p_promo_sk]
- ColumnarToRow
- InputAdapter
- Scan parquet default.promotion [p_promo_sk]
+ Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ InputAdapter
+ BroadcastExchange #12
+ WholeStageCodegen (12)
+ Filter [w_warehouse_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name]
+ InputAdapter
+ BroadcastExchange #13
+ WholeStageCodegen (15)
+ Filter [p_promo_sk]
+ ColumnarToRow
+ InputAdapter
+ Scan parquet default.promotion [p_promo_sk]
InputAdapter
- WholeStageCodegen (17)
+ WholeStageCodegen (19)
Sort [cr_item_sk,cr_order_number]
InputAdapter
- Exchange [cr_item_sk] #12
- WholeStageCodegen (16)
+ Exchange [cr_item_sk,cr_order_number] #14
+ WholeStageCodegen (18)
Project [cr_item_sk,cr_order_number]
Filter [cr_item_sk,cr_order_number]
ColumnarToRow
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
index 3bda562..383b84d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
@@ -706,14 +706,14 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper {
outputPlan match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _,
- DummySparkPlan(_, _, HashPartitioning(leftPartitioningExpressions, _), _, _), _),
+ ShuffleExchangeExec(HashPartitioning(leftPartitioningExpressions, _), _, _), _),
SortExec(_, _,
ShuffleExchangeExec(HashPartitioning(rightPartitioningExpressions, _),
_, _), _), _) =>
assert(leftKeys === smjExec.leftKeys)
assert(rightKeys === smjExec.rightKeys)
- assert(leftPartitioningExpressions == Seq(exprA, exprB, exprA))
- assert(rightPartitioningExpressions == Seq(exprA, exprC, exprA))
+ assert(leftKeys === leftPartitioningExpressions)
+ assert(rightKeys === rightPartitioningExpressions)
case _ => fail(outputPlan.toString)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala
index 046ff78..db99557 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala
@@ -138,6 +138,14 @@ class EnsureRequirementsSuite extends SharedSparkSession {
}
}
+ private def applyEnsureRequirementsWithSubsetKeys(plan: SparkPlan): SparkPlan = {
+ var res: SparkPlan = null
+ withSQLConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false") {
+ res = EnsureRequirements.apply(plan)
+ }
+ res
+ }
+
test("Successful compatibility check with HashShuffleSpec") {
val plan1 = DummySparkPlan(
outputPartitioning = HashPartitioning(exprA :: Nil, 5))
@@ -155,10 +163,14 @@ class EnsureRequirementsSuite extends SharedSparkSession {
case other => fail(other.toString)
}
- // should also work if both partition keys are subset of their corresponding cluster keys
smjExec = SortMergeJoinExec(
exprA :: exprB :: Nil, exprB :: exprC :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ // By default we can't eliminate shuffles if the partitions keys are subset of join keys.
+ assert(EnsureRequirements.apply(smjExec)
+ .collect { case s: ShuffleExchangeLike => s }.length == 2)
+ // with the config set, it should also work if both partition keys are subset of their
+ // corresponding cluster keys
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -169,7 +181,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
smjExec = SortMergeJoinExec(
exprB :: exprA :: Nil, exprC :: exprB :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -186,7 +198,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprA :: exprC :: Nil, 5))
var smjExec = SortMergeJoinExec(
exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -201,7 +213,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprA :: exprC :: exprA :: Nil, 5))
smjExec = SortMergeJoinExec(
exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -216,7 +228,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprA :: exprC :: exprA :: Nil, 5))
smjExec = SortMergeJoinExec(
exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprD :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -231,7 +243,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprA :: exprC :: Nil, 5))
smjExec = SortMergeJoinExec(
exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -249,7 +261,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprD :: Nil, 5))
var smjExec = SortMergeJoinExec(
exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) =>
@@ -266,7 +278,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprD :: Nil, 10))
smjExec = SortMergeJoinExec(
exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) =>
@@ -283,7 +295,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprD :: Nil, 5))
smjExec = SortMergeJoinExec(
exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) =>
@@ -292,8 +304,6 @@ class EnsureRequirementsSuite extends SharedSparkSession {
assert(p.expressions == Seq(exprC))
case other => fail(other.toString)
}
-
-
}
}
@@ -304,7 +314,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprA :: exprC :: exprB :: Nil, 5))
var smjExec = SortMergeJoinExec(
exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) =>
@@ -320,7 +330,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = HashPartitioning(exprA :: exprC :: exprB :: Nil, 5))
smjExec = SortMergeJoinExec(
exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprD :: Nil, Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(leftKeys, rightKeys, _, _,
SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _),
SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) =>
@@ -403,7 +413,10 @@ class EnsureRequirementsSuite extends SharedSparkSession {
}
// HashPartitioning(1) <-> RangePartitioning(10)
- // Only RHS should be shuffled and be converted to HashPartitioning(1) <-> HashPartitioning(1)
+ // If the conf is not set, both sides should be shuffled and be converted to
+ // HashPartitioning(5) <-> HashPartitioning(5)
+ // If the conf is set, only RHS should be shuffled and be converted to
+ // HashPartitioning(1) <-> HashPartitioning(1)
plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA), 1))
plan2 = DummySparkPlan(outputPartitioning = RangePartitioning(
Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 10))
@@ -411,6 +424,16 @@ class EnsureRequirementsSuite extends SharedSparkSession {
exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2)
EnsureRequirements.apply(smjExec) match {
case SortMergeJoinExec(_, _, _, _,
+ SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _),
+ SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) =>
+ assert(left.numPartitions == 5)
+ assert(left.expressions == Seq(exprA, exprB))
+ assert(right.numPartitions == 5)
+ assert(right.expressions == Seq(exprC, exprD))
+ case other => fail(other.toString)
+ }
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
+ case SortMergeJoinExec(_, _, _, _,
SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _),
SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) =>
assert(left.numPartitions == 1)
@@ -446,7 +469,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
smjExec = SortMergeJoinExec(
exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil,
Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(_, _, _, _,
SortExec(_, _, DummySparkPlan(_, _, left: PartitioningCollection, _, _), _),
SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) =>
@@ -463,7 +486,7 @@ class EnsureRequirementsSuite extends SharedSparkSession {
smjExec = SortMergeJoinExec(
exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil,
Inner, None, plan1, plan2)
- EnsureRequirements.apply(smjExec) match {
+ applyEnsureRequirementsWithSubsetKeys(smjExec) match {
case SortMergeJoinExec(_, _, _, _,
SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, right: PartitioningCollection, _, _), _), _) =>
@@ -482,17 +505,17 @@ class EnsureRequirementsSuite extends SharedSparkSession {
// HashPartitioning(5) <-> HashPartitioning(5)
// No shuffle should be inserted
var plan1: SparkPlan = DummySparkPlan(
- outputPartitioning = HashPartitioning(exprA :: Nil, 5))
+ outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5))
var plan2: SparkPlan = DummySparkPlan(
- outputPartitioning = HashPartitioning(exprC :: Nil, 5))
+ outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5))
var smjExec = SortMergeJoinExec(
exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2)
EnsureRequirements.apply(smjExec) match {
case SortMergeJoinExec(_, _, _, _,
SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) =>
- assert(left.expressions === Seq(exprA))
- assert(right.expressions === Seq(exprC))
+ assert(left.expressions === Seq(exprA, exprB))
+ assert(right.expressions === Seq(exprC, exprD))
case other => fail(other.toString)
}
@@ -521,15 +544,15 @@ class EnsureRequirementsSuite extends SharedSparkSession {
outputPartitioning = RangePartitioning(
Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 10))
plan2 = DummySparkPlan(
- outputPartitioning = HashPartitioning(exprD :: Nil, 5))
+ outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5))
smjExec = SortMergeJoinExec(
exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2)
EnsureRequirements.apply(smjExec) match {
case SortMergeJoinExec(_, _, _, _,
SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _),
SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) =>
- assert(left.expressions === Seq(exprB))
- assert(right.expressions === Seq(exprD))
+ assert(left.expressions === Seq(exprA, exprB))
+ assert(right.expressions === Seq(exprC, exprD))
assert(left.numPartitions == 5)
assert(right.numPartitions == 5)
case other => fail(other.toString)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
index be9d1b0..d90c873 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
@@ -773,8 +773,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti
// join predicates is a super set of child's partitioning columns
val bucketedTableTestSpec1 =
- BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))),
- numPartitions = 1, expectedShuffle = false)
+ BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), numPartitions = 1)
testBucketing(
bucketedTableTestSpecLeft = bucketedTableTestSpec1,
bucketedTableTestSpecRight = bucketedTableTestSpec1,
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org