You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2023/02/17 04:48:15 UTC

[spark] branch master updated: [SPARK-39851][SQL] Improve join stats estimation if one side can keep uniqueness

This is an automated email from the ASF dual-hosted git repository.

dongjoon 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 1c0bd1f9f81 [SPARK-39851][SQL] Improve join stats estimation if one side can keep uniqueness
1c0bd1f9f81 is described below

commit 1c0bd1f9f813a341bbfdecb2c5ccde7fbc1bac2d
Author: wankunde <wa...@163.com>
AuthorDate: Thu Feb 16 20:48:02 2023 -0800

    [SPARK-39851][SQL] Improve join stats estimation if one side can keep uniqueness
    
    ### What changes were proposed in this pull request?
    This PR improves join stats estimation if one side can keep uniqueness(The distinct keys of the children of the join are a subset of the join keys). A common case is:
    
    ```sql
    SELECT i_item_sk ss_item_sk
    FROM   item,
           (SELECT DISTINCT iss.i_brand_id    brand_id,
                            iss.i_class_id    class_id,
                            iss.i_category_id category_id
            FROM   item iss) x
    WHERE  i_brand_id = brand_id
           AND i_class_id = class_id
           AND i_category_id = category_id
    ```
    
    In this case, the row count of the join will definitely not expand.
    
    Before this PR:
    
    ```
    == Optimized Logical Plan ==
    Project [i_item_sk#4 AS ss_item_sk#54], Statistics(sizeInBytes=370.8 MiB, rowCount=3.24E+7)
    +- Join Inner, (((i_brand_id#11 = brand_id#51) AND (i_class_id#13 = class_id#52)) AND (i_category_id#15 = category_id#53)), Statistics(sizeInBytes=1112.3 MiB, rowCount=3.24E+7)
       :- Project [i_item_sk#4, i_brand_id#11, i_class_id#13, i_category_id#15], Statistics(sizeInBytes=4.6 MiB, rowCount=2.02E+5)
       :  +- Filter ((isnotnull(i_brand_id#11) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#15)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
       :     +- Relation spark_catalog.default.item[i_item_sk#4,i_item_id#5,i_rec_start_date#6,i_rec_end_date#7,i_item_desc#8,i_current_price#9,i_wholesale_cost#10,i_brand_id#11,i_brand#12,i_class_id#13,i_class#14,i_category_id#15,i_category#16,i_manufact_id#17,i_manufact#18,i_size#19,i_formulation#20,i_color#21,i_units#22,i_container#23,i_manager_id#24,i_product_name#25] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
       +- Aggregate [brand_id#51, class_id#52, category_id#53], [brand_id#51, class_id#52, category_id#53], Statistics(sizeInBytes=2.6 MiB, rowCount=1.37E+5)
          +- Project [i_brand_id#62 AS brand_id#51, i_class_id#64 AS class_id#52, i_category_id#66 AS category_id#53], Statistics(sizeInBytes=3.9 MiB, rowCount=2.02E+5)
             +- Filter ((isnotnull(i_brand_id#62) AND isnotnull(i_class_id#64)) AND isnotnull(i_category_id#66)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
                +- Relation spark_catalog.default.item[i_item_sk#55,i_item_id#56,i_rec_start_date#57,i_rec_end_date#58,i_item_desc#59,i_current_price#60,i_wholesale_cost#61,i_brand_id#62,i_brand#63,i_class_id#64,i_class#65,i_category_id#66,i_category#67,i_manufact_id#68,i_manufact#69,i_size#70,i_formulation#71,i_color#72,i_units#73,i_container#74,i_manager_id#75,i_product_name#76] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
    ```
    
    After this PR:
    
    ```
    == Optimized Logical Plan ==
    Project [i_item_sk#4 AS ss_item_sk#54], Statistics(sizeInBytes=2.3 MiB, rowCount=2.02E+5)
    +- Join Inner, (((i_brand_id#11 = brand_id#51) AND (i_class_id#13 = class_id#52)) AND (i_category_id#15 = category_id#53)), Statistics(sizeInBytes=7.0 MiB, rowCount=2.02E+5)
       :- Project [i_item_sk#4, i_brand_id#11, i_class_id#13, i_category_id#15], Statistics(sizeInBytes=4.6 MiB, rowCount=2.02E+5)
       :  +- Filter ((isnotnull(i_brand_id#11) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#15)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
       :     +- Relation spark_catalog.default.item[i_item_sk#4,i_item_id#5,i_rec_start_date#6,i_rec_end_date#7,i_item_desc#8,i_current_price#9,i_wholesale_cost#10,i_brand_id#11,i_brand#12,i_class_id#13,i_class#14,i_category_id#15,i_category#16,i_manufact_id#17,i_manufact#18,i_size#19,i_formulation#20,i_color#21,i_units#22,i_container#23,i_manager_id#24,i_product_name#25] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
       +- Aggregate [brand_id#51, class_id#52, category_id#53], [brand_id#51, class_id#52, category_id#53], Statistics(sizeInBytes=2.6 MiB, rowCount=1.37E+5)
          +- Project [i_brand_id#62 AS brand_id#51, i_class_id#64 AS class_id#52, i_category_id#66 AS category_id#53], Statistics(sizeInBytes=3.9 MiB, rowCount=2.02E+5)
             +- Filter ((isnotnull(i_brand_id#62) AND isnotnull(i_class_id#64)) AND isnotnull(i_category_id#66)), Statistics(sizeInBytes=84.6 MiB, rowCount=2.02E+5)
                +- Relation spark_catalog.default.item[i_item_sk#55,i_item_id#56,i_rec_start_date#57,i_rec_end_date#58,i_item_desc#59,i_current_price#60,i_wholesale_cost#61,i_brand_id#62,i_brand#63,i_class_id#64,i_class#65,i_category_id#66,i_category#67,i_manufact_id#68,i_manufact#69,i_size#70,i_formulation#71,i_color#72,i_units#73,i_container#74,i_manager_id#75,i_product_name#76] parquet, Statistics(sizeInBytes=85.2 MiB, rowCount=2.04E+5)
    ```
    
    ### Why are the changes needed?
    Plan more broadcast joins to improve query performance.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    Unit test and TPC-DS benchmark test.
    
    SQL     Before this PR(Seconds) After this PR(Seconds)
    q14a    187     164
    
    Closes #39923 from wankunde/SPARK-39851.
    
    Lead-authored-by: wankunde <wa...@163.com>
    Co-authored-by: Kun Wan <wa...@apache.org>
    Signed-off-by: Dongjoon Hyun <do...@apache.org>
---
 .../logical/statsEstimation/JoinEstimation.scala   |  22 +-
 .../SizeInBytesOnlyStatsPlanVisitor.scala          |  15 +-
 .../BasicStatsEstimationSuite.scala                |  33 +-
 .../approved-plans-v1_4/q14a.sf100/explain.txt     | 678 ++++++++----------
 .../approved-plans-v1_4/q14a.sf100/simplified.txt  | 328 ++++-----
 .../approved-plans-v1_4/q14b.sf100/explain.txt     | 631 ++++++++--------
 .../approved-plans-v1_4/q14b.sf100/simplified.txt  | 335 ++++-----
 .../approved-plans-v2_7/q14.sf100/explain.txt      | 631 ++++++++--------
 .../approved-plans-v2_7/q14.sf100/simplified.txt   | 335 ++++-----
 .../approved-plans-v2_7/q14a.sf100/explain.txt     | 790 ++++++++++-----------
 .../approved-plans-v2_7/q14a.sf100/simplified.txt  | 358 +++++-----
 11 files changed, 1936 insertions(+), 2220 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
index 30f10970d1a..c6e76df1b31 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
@@ -21,7 +21,7 @@ import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression, ExpressionSet}
 import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Histogram, Join, Statistics}
@@ -56,10 +56,13 @@ case class JoinEstimation(join: Join) extends Logging {
     case _ if !rowCountsExist(join.left, join.right) =>
       None
 
-    case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, _, _, _) =>
+    case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, _) =>
       // 1. Compute join selectivity
       val joinKeyPairs = extractJoinKeysWithColStats(leftKeys, rightKeys)
-      val (numInnerJoinedRows, keyStatsAfterJoin) = computeCardinalityAndStats(joinKeyPairs)
+      val leftSideUniqueness = left.distinctKeys.exists(_.subsetOf(ExpressionSet(leftKeys)))
+      val rightSideUniqueness = right.distinctKeys.exists(_.subsetOf(ExpressionSet(rightKeys)))
+      val (numInnerJoinedRows, keyStatsAfterJoin) =
+        computeCardinalityAndStats(joinKeyPairs, leftSideUniqueness, rightSideUniqueness)
 
       // 2. Estimate the number of output rows
       val leftRows = leftStats.rowCount.get
@@ -175,10 +178,17 @@ case class JoinEstimation(join: Join) extends Logging {
    * @return join cardinality, and column stats for join keys after the join
    */
   // scalastyle:on
-  private def computeCardinalityAndStats(keyPairs: Seq[(AttributeReference, AttributeReference)])
-    : (BigInt, AttributeMap[ColumnStat]) = {
+  private def computeCardinalityAndStats(
+      keyPairs: Seq[(AttributeReference, AttributeReference)],
+      leftSideUniqueness: Boolean,
+      rightSideUniqueness: Boolean): (BigInt, AttributeMap[ColumnStat]) = {
     // If there's no column stats available for join keys, estimate as cartesian product.
-    var joinCard: BigInt = leftStats.rowCount.get * rightStats.rowCount.get
+    var joinCard: BigInt = (leftSideUniqueness, rightSideUniqueness) match {
+      case (true, true) => leftStats.rowCount.get.min(rightStats.rowCount.get)
+      case (true, false) => rightStats.rowCount.get
+      case (false, true) => leftStats.rowCount.get
+      case _ => leftStats.rowCount.get * rightStats.rowCount.get
+    }
     val keyStatsAfterJoin = new mutable.HashMap[Attribute, ColumnStat]()
     var i = 0
     while(i < keyPairs.length && joinCard != 0) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
index 77c728ba7c5..14cce02d99b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
@@ -17,8 +17,9 @@
 
 package org.apache.spark.sql.catalyst.plans.logical.statsEstimation
 
-import org.apache.spark.sql.catalyst.expressions.AttributeMap
-import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi}
+import org.apache.spark.sql.catalyst.expressions.{AttributeMap, ExpressionSet}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
 
 /**
@@ -112,6 +113,16 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] {
       case LeftAnti | LeftSemi =>
         // LeftSemi and LeftAnti won't ever be bigger than left
         p.left.stats
+      case Inner | LeftOuter | RightOuter | FullOuter =>
+        p match {
+          case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, left, right, _)
+              if left.distinctKeys.exists(_.subsetOf(ExpressionSet(leftKeys))) ||
+                right.distinctKeys.exists(_.subsetOf(ExpressionSet(rightKeys))) =>
+            // The sizeInBytes should be > 1 because sizeInBytes * 1 != sizeInBytes + 1.
+            Statistics(sizeInBytes = p.children.map(_.stats.sizeInBytes).filter(_ > 1L).sum)
+          case _ =>
+            default(p)
+        }
       case _ =>
         default(p)
     }
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
index 4362e0c5172..33e521eb65a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
 import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeMap, AttributeReference, Literal, SortOrder}
-import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.connector.catalog.SupportsNamespaces
 import org.apache.spark.sql.internal.SQLConf
@@ -328,6 +328,37 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase {
       expectedStatsCboOff = Statistics(sizeInBytes = sizeInBytes))
   }
 
+  test("SPARK-39851: Improve join stats estimation if one side can keep uniqueness") {
+    val brandId = attr("brand_id")
+    val classId = attr("class_id")
+    val aliasedBrandId = brandId.as("new_brand_id")
+    val aliasedClassId = classId.as("new_class_id")
+
+    val tableSize = 4059900
+    val tableRowCnt = 202995
+
+    val tbl = StatsTestPlan(
+      outputList = Seq(brandId, classId),
+      size = Some(tableSize),
+      rowCount = tableRowCnt,
+      attributeStats =
+        AttributeMap(Seq(
+          brandId -> ColumnStat(Some(858), Some(101001), Some(1016017), Some(0), Some(4), Some(4)),
+          classId -> ColumnStat(Some(16), Some(1), Some(16), Some(0), Some(4), Some(4)))))
+
+    val join = Join(
+      tbl,
+      tbl.groupBy(brandId, classId)(aliasedBrandId, aliasedClassId),
+      Inner,
+      Some(brandId === aliasedBrandId.toAttribute && classId === aliasedClassId.toAttribute),
+      JoinHint.NONE)
+
+    checkStats(
+      join,
+      expectedStatsCboOn = Statistics(4871880, Some(tableRowCnt), join.stats.attributeStats),
+      expectedStatsCboOff = Statistics(sizeInBytes = 4059900 * 2))
+  }
+
   test("row size and column stats estimation for sort") {
     val columnInfo = AttributeMap(
       Seq(
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
index 8b962208051..e350459c60c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
@@ -1,127 +1,115 @@
 == Physical Plan ==
-TakeOrderedAndProject (123)
-+- * HashAggregate (122)
-   +- Exchange (121)
-      +- * HashAggregate (120)
-         +- * Expand (119)
-            +- Union (118)
-               :- * Project (79)
-               :  +- * Filter (78)
-               :     +- * HashAggregate (77)
-               :        +- Exchange (76)
-               :           +- * HashAggregate (75)
-               :              +- * Project (74)
-               :                 +- * BroadcastHashJoin Inner BuildRight (73)
-               :                    :- * Project (63)
-               :                    :  +- * BroadcastHashJoin Inner BuildRight (62)
-               :                    :     :- * SortMergeJoin LeftSemi (60)
-               :                    :     :  :- * Sort (5)
-               :                    :     :  :  +- Exchange (4)
-               :                    :     :  :     +- * Filter (3)
-               :                    :     :  :        +- * ColumnarToRow (2)
-               :                    :     :  :           +- Scan parquet spark_catalog.default.store_sales (1)
-               :                    :     :  +- * Sort (59)
-               :                    :     :     +- Exchange (58)
-               :                    :     :        +- * Project (57)
-               :                    :     :           +- * BroadcastHashJoin Inner BuildRight (56)
-               :                    :     :              :- * Filter (8)
-               :                    :     :              :  +- * ColumnarToRow (7)
-               :                    :     :              :     +- Scan parquet spark_catalog.default.item (6)
-               :                    :     :              +- BroadcastExchange (55)
-               :                    :     :                 +- * SortMergeJoin LeftSemi (54)
-               :                    :     :                    :- * Sort (42)
-               :                    :     :                    :  +- Exchange (41)
-               :                    :     :                    :     +- * HashAggregate (40)
-               :                    :     :                    :        +- Exchange (39)
-               :                    :     :                    :           +- * HashAggregate (38)
-               :                    :     :                    :              +- * Project (37)
-               :                    :     :                    :                 +- * BroadcastHashJoin Inner BuildRight (36)
-               :                    :     :                    :                    :- * Project (14)
-               :                    :     :                    :                    :  +- * BroadcastHashJoin Inner BuildRight (13)
-               :                    :     :                    :                    :     :- * Filter (11)
-               :                    :     :                    :                    :     :  +- * ColumnarToRow (10)
-               :                    :     :                    :                    :     :     +- Scan parquet spark_catalog.default.store_sales (9)
-               :                    :     :                    :                    :     +- ReusedExchange (12)
-               :                    :     :                    :                    +- BroadcastExchange (35)
-               :                    :     :                    :                       +- * SortMergeJoin LeftSemi (34)
-               :                    :     :                    :                          :- * Sort (19)
-               :                    :     :                    :                          :  +- Exchange (18)
-               :                    :     :                    :                          :     +- * Filter (17)
-               :                    :     :                    :                          :        +- * ColumnarToRow (16)
-               :                    :     :                    :                          :           +- Scan parquet spark_catalog.default.item (15)
-               :                    :     :                    :                          +- * Sort (33)
-               :                    :     :                    :                             +- Exchange (32)
-               :                    :     :                    :                                +- * Project (31)
-               :                    :     :                    :                                   +- * BroadcastHashJoin Inner BuildRight (30)
-               :                    :     :                    :                                      :- * Project (25)
-               :                    :     :                    :                                      :  +- * BroadcastHashJoin Inner BuildRight (24)
-               :                    :     :                    :                                      :     :- * Filter (22)
-               :                    :     :                    :                                      :     :  +- * ColumnarToRow (21)
-               :                    :     :                    :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (20)
-               :                    :     :                    :                                      :     +- ReusedExchange (23)
-               :                    :     :                    :                                      +- BroadcastExchange (29)
-               :                    :     :                    :                                         +- * Filter (28)
-               :                    :     :                    :                                            +- * ColumnarToRow (27)
-               :                    :     :                    :                                               +- Scan parquet spark_catalog.default.item (26)
-               :                    :     :                    +- * Sort (53)
-               :                    :     :                       +- Exchange (52)
-               :                    :     :                          +- * Project (51)
-               :                    :     :                             +- * BroadcastHashJoin Inner BuildRight (50)
-               :                    :     :                                :- * Project (48)
-               :                    :     :                                :  +- * BroadcastHashJoin Inner BuildRight (47)
-               :                    :     :                                :     :- * Filter (45)
-               :                    :     :                                :     :  +- * ColumnarToRow (44)
-               :                    :     :                                :     :     +- Scan parquet spark_catalog.default.web_sales (43)
-               :                    :     :                                :     +- ReusedExchange (46)
-               :                    :     :                                +- ReusedExchange (49)
-               :                    :     +- ReusedExchange (61)
-               :                    +- BroadcastExchange (72)
-               :                       +- * SortMergeJoin LeftSemi (71)
-               :                          :- * Sort (68)
-               :                          :  +- Exchange (67)
-               :                          :     +- * Filter (66)
-               :                          :        +- * ColumnarToRow (65)
-               :                          :           +- Scan parquet spark_catalog.default.item (64)
-               :                          +- * Sort (70)
-               :                             +- ReusedExchange (69)
-               :- * Project (98)
-               :  +- * Filter (97)
-               :     +- * HashAggregate (96)
-               :        +- Exchange (95)
-               :           +- * HashAggregate (94)
-               :              +- * Project (93)
-               :                 +- * BroadcastHashJoin Inner BuildRight (92)
-               :                    :- * Project (90)
-               :                    :  +- * BroadcastHashJoin Inner BuildRight (89)
-               :                    :     :- * SortMergeJoin LeftSemi (87)
-               :                    :     :  :- * Sort (84)
-               :                    :     :  :  +- Exchange (83)
-               :                    :     :  :     +- * Filter (82)
-               :                    :     :  :        +- * ColumnarToRow (81)
-               :                    :     :  :           +- Scan parquet spark_catalog.default.catalog_sales (80)
-               :                    :     :  +- * Sort (86)
-               :                    :     :     +- ReusedExchange (85)
-               :                    :     +- ReusedExchange (88)
-               :                    +- ReusedExchange (91)
-               +- * Project (117)
-                  +- * Filter (116)
-                     +- * HashAggregate (115)
-                        +- Exchange (114)
-                           +- * HashAggregate (113)
-                              +- * Project (112)
-                                 +- * BroadcastHashJoin Inner BuildRight (111)
-                                    :- * Project (109)
-                                    :  +- * BroadcastHashJoin Inner BuildRight (108)
-                                    :     :- * SortMergeJoin LeftSemi (106)
-                                    :     :  :- * Sort (103)
-                                    :     :  :  +- Exchange (102)
-                                    :     :  :     +- * Filter (101)
-                                    :     :  :        +- * ColumnarToRow (100)
-                                    :     :  :           +- Scan parquet spark_catalog.default.web_sales (99)
-                                    :     :  +- * Sort (105)
-                                    :     :     +- ReusedExchange (104)
-                                    :     +- ReusedExchange (107)
-                                    +- ReusedExchange (110)
+TakeOrderedAndProject (111)
++- * HashAggregate (110)
+   +- Exchange (109)
+      +- * HashAggregate (108)
+         +- * Expand (107)
+            +- Union (106)
+               :- * Project (73)
+               :  +- * Filter (72)
+               :     +- * HashAggregate (71)
+               :        +- Exchange (70)
+               :           +- * HashAggregate (69)
+               :              +- * Project (68)
+               :                 +- * BroadcastHashJoin Inner BuildRight (67)
+               :                    :- * Project (60)
+               :                    :  +- * BroadcastHashJoin Inner BuildRight (59)
+               :                    :     :- * BroadcastHashJoin LeftSemi BuildRight (57)
+               :                    :     :  :- * Filter (3)
+               :                    :     :  :  +- * ColumnarToRow (2)
+               :                    :     :  :     +- Scan parquet spark_catalog.default.store_sales (1)
+               :                    :     :  +- BroadcastExchange (56)
+               :                    :     :     +- * Project (55)
+               :                    :     :        +- * BroadcastHashJoin Inner BuildRight (54)
+               :                    :     :           :- * Filter (6)
+               :                    :     :           :  +- * ColumnarToRow (5)
+               :                    :     :           :     +- Scan parquet spark_catalog.default.item (4)
+               :                    :     :           +- BroadcastExchange (53)
+               :                    :     :              +- * SortMergeJoin LeftSemi (52)
+               :                    :     :                 :- * Sort (40)
+               :                    :     :                 :  +- Exchange (39)
+               :                    :     :                 :     +- * HashAggregate (38)
+               :                    :     :                 :        +- Exchange (37)
+               :                    :     :                 :           +- * HashAggregate (36)
+               :                    :     :                 :              +- * Project (35)
+               :                    :     :                 :                 +- * BroadcastHashJoin Inner BuildRight (34)
+               :                    :     :                 :                    :- * Project (12)
+               :                    :     :                 :                    :  +- * BroadcastHashJoin Inner BuildRight (11)
+               :                    :     :                 :                    :     :- * Filter (9)
+               :                    :     :                 :                    :     :  +- * ColumnarToRow (8)
+               :                    :     :                 :                    :     :     +- Scan parquet spark_catalog.default.store_sales (7)
+               :                    :     :                 :                    :     +- ReusedExchange (10)
+               :                    :     :                 :                    +- BroadcastExchange (33)
+               :                    :     :                 :                       +- * SortMergeJoin LeftSemi (32)
+               :                    :     :                 :                          :- * Sort (17)
+               :                    :     :                 :                          :  +- Exchange (16)
+               :                    :     :                 :                          :     +- * Filter (15)
+               :                    :     :                 :                          :        +- * ColumnarToRow (14)
+               :                    :     :                 :                          :           +- Scan parquet spark_catalog.default.item (13)
+               :                    :     :                 :                          +- * Sort (31)
+               :                    :     :                 :                             +- Exchange (30)
+               :                    :     :                 :                                +- * Project (29)
+               :                    :     :                 :                                   +- * BroadcastHashJoin Inner BuildRight (28)
+               :                    :     :                 :                                      :- * Project (23)
+               :                    :     :                 :                                      :  +- * BroadcastHashJoin Inner BuildRight (22)
+               :                    :     :                 :                                      :     :- * Filter (20)
+               :                    :     :                 :                                      :     :  +- * ColumnarToRow (19)
+               :                    :     :                 :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (18)
+               :                    :     :                 :                                      :     +- ReusedExchange (21)
+               :                    :     :                 :                                      +- BroadcastExchange (27)
+               :                    :     :                 :                                         +- * Filter (26)
+               :                    :     :                 :                                            +- * ColumnarToRow (25)
+               :                    :     :                 :                                               +- Scan parquet spark_catalog.default.item (24)
+               :                    :     :                 +- * Sort (51)
+               :                    :     :                    +- Exchange (50)
+               :                    :     :                       +- * Project (49)
+               :                    :     :                          +- * BroadcastHashJoin Inner BuildRight (48)
+               :                    :     :                             :- * Project (46)
+               :                    :     :                             :  +- * BroadcastHashJoin Inner BuildRight (45)
+               :                    :     :                             :     :- * Filter (43)
+               :                    :     :                             :     :  +- * ColumnarToRow (42)
+               :                    :     :                             :     :     +- Scan parquet spark_catalog.default.web_sales (41)
+               :                    :     :                             :     +- ReusedExchange (44)
+               :                    :     :                             +- ReusedExchange (47)
+               :                    :     +- ReusedExchange (58)
+               :                    +- BroadcastExchange (66)
+               :                       +- * BroadcastHashJoin LeftSemi BuildRight (65)
+               :                          :- * Filter (63)
+               :                          :  +- * ColumnarToRow (62)
+               :                          :     +- Scan parquet spark_catalog.default.item (61)
+               :                          +- ReusedExchange (64)
+               :- * Project (89)
+               :  +- * Filter (88)
+               :     +- * HashAggregate (87)
+               :        +- Exchange (86)
+               :           +- * HashAggregate (85)
+               :              +- * Project (84)
+               :                 +- * BroadcastHashJoin Inner BuildRight (83)
+               :                    :- * Project (81)
+               :                    :  +- * BroadcastHashJoin Inner BuildRight (80)
+               :                    :     :- * BroadcastHashJoin LeftSemi BuildRight (78)
+               :                    :     :  :- * Filter (76)
+               :                    :     :  :  +- * ColumnarToRow (75)
+               :                    :     :  :     +- Scan parquet spark_catalog.default.catalog_sales (74)
+               :                    :     :  +- ReusedExchange (77)
+               :                    :     +- ReusedExchange (79)
+               :                    +- ReusedExchange (82)
+               +- * Project (105)
+                  +- * Filter (104)
+                     +- * HashAggregate (103)
+                        +- Exchange (102)
+                           +- * HashAggregate (101)
+                              +- * Project (100)
+                                 +- * BroadcastHashJoin Inner BuildRight (99)
+                                    :- * Project (97)
+                                    :  +- * BroadcastHashJoin Inner BuildRight (96)
+                                    :     :- * BroadcastHashJoin LeftSemi BuildRight (94)
+                                    :     :  :- * Filter (92)
+                                    :     :  :  +- * ColumnarToRow (91)
+                                    :     :  :     +- Scan parquet spark_catalog.default.web_sales (90)
+                                    :     :  +- ReusedExchange (93)
+                                    :     +- ReusedExchange (95)
+                                    +- ReusedExchange (98)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -132,36 +120,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 Condition : isnotnull(ss_item_sk#1)
 
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
 
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
 Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Batched: true
 Location: InMemoryFileIndex []
@@ -169,49 +149,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int>
 
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Condition : isnotnull(ss_item_sk#10)
 
-(12) ReusedExchange [Reuses operator id: 152]
+(10) ReusedExchange [Reuses operator id: 140]
 Output [1]: [d_date_sk#13]
 
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_sold_date_sk#11]
 Right keys [1]: [d_date_sk#13]
 Join type: Inner
 Join condition: None
 
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
 Output [1]: [ss_item_sk#10]
 Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
 
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
 
-(18) Exchange
+(16) Exchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
 
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
 
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
 Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Batched: true
 Location: InMemoryFileIndex []
@@ -219,109 +199,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int>
 
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Condition : isnotnull(cs_item_sk#18)
 
-(23) ReusedExchange [Reuses operator id: 152]
+(21) ReusedExchange [Reuses operator id: 140]
 Output [1]: [d_date_sk#20]
 
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_sold_date_sk#19]
 Right keys [1]: [d_date_sk#20]
 Join type: Inner
 Join condition: None
 
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
 Output [1]: [cs_item_sk#18]
 Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
 
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Condition : isnotnull(i_item_sk#21)
 
-(29) BroadcastExchange
+(27) BroadcastExchange
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
 
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_item_sk#18]
 Right keys [1]: [i_item_sk#21]
 Join type: Inner
 Join condition: None
 
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
 Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(32) Exchange
+(30) Exchange
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
 
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
 
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
 Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
 Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
 Join type: LeftSemi
 Join condition: None
 
-(35) BroadcastExchange
+(33) BroadcastExchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_item_sk#10]
 Right keys [1]: [i_item_sk#14]
 Join type: Inner
 Join condition: None
 
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
 Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
 Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(39) Exchange
+(37) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(41) Exchange
+(39) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
 
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
 Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Batched: true
 Location: InMemoryFileIndex []
@@ -329,170 +309,154 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int>
 
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Condition : isnotnull(ws_item_sk#28)
 
-(46) ReusedExchange [Reuses operator id: 152]
+(44) ReusedExchange [Reuses operator id: 140]
 Output [1]: [d_date_sk#30]
 
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_sold_date_sk#29]
 Right keys [1]: [d_date_sk#30]
 Join type: Inner
 Join condition: None
 
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
 Output [1]: [ws_item_sk#28]
 Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
 
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
 Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_item_sk#28]
 Right keys [1]: [i_item_sk#31]
 Join type: Inner
 Join condition: None
 
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
 Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(52) Exchange
+(50) Exchange
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
 
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
 
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
 Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
 Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
 Join type: LeftSemi
 Join condition: None
 
-(55) BroadcastExchange
+(53) BroadcastExchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
 
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
 Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
 Right keys [3]: [brand_id#25, class_id#26, category_id#27]
 Join type: Inner
 Join condition: None
 
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
 Output [1]: [i_item_sk#6 AS ss_item_sk#35]
 Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
 
-(58) Exchange
-Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
-
-(59) Sort [codegen id : 20]
+(56) BroadcastExchange
 Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
 
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(61) ReusedExchange [Reuses operator id: 147]
+(58) ReusedExchange [Reuses operator id: 135]
 Output [1]: [d_date_sk#36]
 
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_sold_date_sk#4]
 Right keys [1]: [d_date_sk#36]
 Join type: Inner
 Join condition: None
 
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
 Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
 Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
 
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Condition : isnotnull(i_item_sk#37)
 
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
 Left keys [1]: [i_item_sk#37]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(72) BroadcastExchange
+(66) BroadcastExchange
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
 
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [i_item_sk#37]
 Join type: Inner
 Join condition: None
 
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
 Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
-(76) Exchange
+(70) Exchange
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
-(77) HashAggregate [codegen id : 44]
+(71) HashAggregate [codegen id : 38]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
 Results [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50]
 
-(78) Filter [codegen id : 44]
+(72) Filter [codegen id : 38]
 Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50]
 Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6))))
 
-(79) Project [codegen id : 44]
+(73) Project [codegen id : 38]
 Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#38 AS i_brand_id#54, i_class_id#39 AS i_class_id#55, i_category_id#40 AS i_category_id#56]
 Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50]
 
-(80) Scan parquet spark_catalog.default.catalog_sales
+(74) Scan parquet spark_catalog.default.catalog_sales
 Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
 Batched: true
 Location: InMemoryFileIndex []
@@ -500,87 +464,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)>
 
-(81) ColumnarToRow [codegen id : 45]
+(75) ColumnarToRow [codegen id : 75]
 Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
 
-(82) Filter [codegen id : 45]
+(76) Filter [codegen id : 75]
 Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
 Condition : isnotnull(cs_item_sk#57)
 
-(83) Exchange
-Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
-Arguments: hashpartitioning(cs_item_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(84) Sort [codegen id : 46]
-Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60]
-Arguments: [cs_item_sk#57 ASC NULLS FIRST], false, 0
-
-(85) ReusedExchange [Reuses operator id: 58]
+(77) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(86) Sort [codegen id : 64]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(87) SortMergeJoin [codegen id : 87]
+(78) BroadcastHashJoin [codegen id : 75]
 Left keys [1]: [cs_item_sk#57]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(88) ReusedExchange [Reuses operator id: 147]
+(79) ReusedExchange [Reuses operator id: 135]
 Output [1]: [d_date_sk#61]
 
-(89) BroadcastHashJoin [codegen id : 87]
+(80) BroadcastHashJoin [codegen id : 75]
 Left keys [1]: [cs_sold_date_sk#60]
 Right keys [1]: [d_date_sk#61]
 Join type: Inner
 Join condition: None
 
-(90) Project [codegen id : 87]
+(81) Project [codegen id : 75]
 Output [3]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59]
 Input [5]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, d_date_sk#61]
 
-(91) ReusedExchange [Reuses operator id: 72]
+(82) ReusedExchange [Reuses operator id: 66]
 Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65]
 
-(92) BroadcastHashJoin [codegen id : 87]
+(83) BroadcastHashJoin [codegen id : 75]
 Left keys [1]: [cs_item_sk#57]
 Right keys [1]: [i_item_sk#62]
 Join type: Inner
 Join condition: None
 
-(93) Project [codegen id : 87]
+(84) Project [codegen id : 75]
 Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65]
 Input [7]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65]
 
-(94) HashAggregate [codegen id : 87]
+(85) HashAggregate [codegen id : 75]
 Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65]
 Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65]
 Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)]
 Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68]
 Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71]
 
-(95) Exchange
+(86) Exchange
 Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71]
-Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
-(96) HashAggregate [codegen id : 88]
+(87) HashAggregate [codegen id : 76]
 Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#69, isEmpty#70, count#71]
 Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65]
 Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)]
 Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73]
 Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75]
 
-(97) Filter [codegen id : 88]
+(88) Filter [codegen id : 76]
 Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75]
 Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6))))
 
-(98) Project [codegen id : 88]
+(89) Project [codegen id : 76]
 Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#63, i_class_id#64, i_category_id#65]
 Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#74, number_sales#75]
 
-(99) Scan parquet spark_catalog.default.web_sales
+(90) Scan parquet spark_catalog.default.web_sales
 Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
 Batched: true
 Location: InMemoryFileIndex []
@@ -588,303 +540,291 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)>
 
-(100) ColumnarToRow [codegen id : 89]
+(91) ColumnarToRow [codegen id : 113]
 Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
 
-(101) Filter [codegen id : 89]
+(92) Filter [codegen id : 113]
 Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
 Condition : isnotnull(ws_item_sk#77)
 
-(102) Exchange
-Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
-Arguments: hashpartitioning(ws_item_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=16]
-
-(103) Sort [codegen id : 90]
-Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80]
-Arguments: [ws_item_sk#77 ASC NULLS FIRST], false, 0
-
-(104) ReusedExchange [Reuses operator id: 58]
+(93) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(105) Sort [codegen id : 108]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(106) SortMergeJoin [codegen id : 131]
+(94) BroadcastHashJoin [codegen id : 113]
 Left keys [1]: [ws_item_sk#77]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(107) ReusedExchange [Reuses operator id: 147]
+(95) ReusedExchange [Reuses operator id: 135]
 Output [1]: [d_date_sk#81]
 
-(108) BroadcastHashJoin [codegen id : 131]
+(96) BroadcastHashJoin [codegen id : 113]
 Left keys [1]: [ws_sold_date_sk#80]
 Right keys [1]: [d_date_sk#81]
 Join type: Inner
 Join condition: None
 
-(109) Project [codegen id : 131]
+(97) Project [codegen id : 113]
 Output [3]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79]
 Input [5]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, d_date_sk#81]
 
-(110) ReusedExchange [Reuses operator id: 72]
+(98) ReusedExchange [Reuses operator id: 66]
 Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85]
 
-(111) BroadcastHashJoin [codegen id : 131]
+(99) BroadcastHashJoin [codegen id : 113]
 Left keys [1]: [ws_item_sk#77]
 Right keys [1]: [i_item_sk#82]
 Join type: Inner
 Join condition: None
 
-(112) Project [codegen id : 131]
+(100) Project [codegen id : 113]
 Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85]
 Input [7]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85]
 
-(113) HashAggregate [codegen id : 131]
+(101) HashAggregate [codegen id : 113]
 Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85]
 Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85]
 Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)]
 Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88]
 Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91]
 
-(114) Exchange
+(102) Exchange
 Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91]
-Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=13]
 
-(115) HashAggregate [codegen id : 132]
+(103) HashAggregate [codegen id : 114]
 Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#89, isEmpty#90, count#91]
 Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85]
 Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93]
 Results [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95]
 
-(116) Filter [codegen id : 132]
+(104) Filter [codegen id : 114]
 Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95]
 Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6))))
 
-(117) Project [codegen id : 132]
+(105) Project [codegen id : 114]
 Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#83, i_class_id#84, i_category_id#85]
 Input [5]: [i_brand_id#83, i_class_id#84, i_category_id#85, sales#94, number_sales#95]
 
-(118) Union
+(106) Union
 
-(119) Expand [codegen id : 133]
+(107) Expand [codegen id : 115]
 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56]
 Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
 
-(120) HashAggregate [codegen id : 133]
+(108) HashAggregate [codegen id : 115]
 Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
 Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
 Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)]
 Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104]
 Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107]
 
-(121) Exchange
+(109) Exchange
 Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107]
-Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=18]
+Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=14]
 
-(122) HashAggregate [codegen id : 134]
+(110) HashAggregate [codegen id : 116]
 Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107]
 Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101]
 Functions [2]: [sum(sales#49), sum(number_sales#50)]
 Aggregate Attributes [2]: [sum(sales#49)#108, sum(number_sales#50)#109]
 Results [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales#49)#108 AS sum(sales)#110, sum(number_sales#50)#109 AS sum(number_sales)#111]
 
-(123) TakeOrderedAndProject
+(111) TakeOrderedAndProject
 Input [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111]
 Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_class_id#99 ASC NULLS FIRST, i_category_id#100 ASC NULLS FIRST], [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111]
 
 ===== Subqueries =====
 
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#51, [id=#52]
-* HashAggregate (142)
-+- Exchange (141)
-   +- * HashAggregate (140)
-      +- Union (139)
-         :- * Project (128)
-         :  +- * BroadcastHashJoin Inner BuildRight (127)
-         :     :- * ColumnarToRow (125)
-         :     :  +- Scan parquet spark_catalog.default.store_sales (124)
-         :     +- ReusedExchange (126)
-         :- * Project (133)
-         :  +- * BroadcastHashJoin Inner BuildRight (132)
-         :     :- * ColumnarToRow (130)
-         :     :  +- Scan parquet spark_catalog.default.catalog_sales (129)
-         :     +- ReusedExchange (131)
-         +- * Project (138)
-            +- * BroadcastHashJoin Inner BuildRight (137)
-               :- * ColumnarToRow (135)
-               :  +- Scan parquet spark_catalog.default.web_sales (134)
-               +- ReusedExchange (136)
-
-
-(124) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#51, [id=#52]
+* HashAggregate (130)
++- Exchange (129)
+   +- * HashAggregate (128)
+      +- Union (127)
+         :- * Project (116)
+         :  +- * BroadcastHashJoin Inner BuildRight (115)
+         :     :- * ColumnarToRow (113)
+         :     :  +- Scan parquet spark_catalog.default.store_sales (112)
+         :     +- ReusedExchange (114)
+         :- * Project (121)
+         :  +- * BroadcastHashJoin Inner BuildRight (120)
+         :     :- * ColumnarToRow (118)
+         :     :  +- Scan parquet spark_catalog.default.catalog_sales (117)
+         :     +- ReusedExchange (119)
+         +- * Project (126)
+            +- * BroadcastHashJoin Inner BuildRight (125)
+               :- * ColumnarToRow (123)
+               :  +- Scan parquet spark_catalog.default.web_sales (122)
+               +- ReusedExchange (124)
+
+
+(112) Scan parquet spark_catalog.default.store_sales
 Output [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)]
 ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(125) ColumnarToRow [codegen id : 2]
+(113) ColumnarToRow [codegen id : 2]
 Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114]
 
-(126) ReusedExchange [Reuses operator id: 152]
+(114) ReusedExchange [Reuses operator id: 140]
 Output [1]: [d_date_sk#115]
 
-(127) BroadcastHashJoin [codegen id : 2]
+(115) BroadcastHashJoin [codegen id : 2]
 Left keys [1]: [ss_sold_date_sk#114]
 Right keys [1]: [d_date_sk#115]
 Join type: Inner
 Join condition: None
 
-(128) Project [codegen id : 2]
+(116) Project [codegen id : 2]
 Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117]
 Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115]
 
-(129) Scan parquet spark_catalog.default.catalog_sales
+(117) Scan parquet spark_catalog.default.catalog_sales
 Output [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)]
 ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
 
-(130) ColumnarToRow [codegen id : 4]
+(118) ColumnarToRow [codegen id : 4]
 Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120]
 
-(131) ReusedExchange [Reuses operator id: 152]
+(119) ReusedExchange [Reuses operator id: 140]
 Output [1]: [d_date_sk#121]
 
-(132) BroadcastHashJoin [codegen id : 4]
+(120) BroadcastHashJoin [codegen id : 4]
 Left keys [1]: [cs_sold_date_sk#120]
 Right keys [1]: [d_date_sk#121]
 Join type: Inner
 Join condition: None
 
-(133) Project [codegen id : 4]
+(121) Project [codegen id : 4]
 Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123]
 Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121]
 
-(134) Scan parquet spark_catalog.default.web_sales
+(122) Scan parquet spark_catalog.default.web_sales
 Output [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)]
 ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
 
-(135) ColumnarToRow [codegen id : 6]
+(123) ColumnarToRow [codegen id : 6]
 Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126]
 
-(136) ReusedExchange [Reuses operator id: 152]
+(124) ReusedExchange [Reuses operator id: 140]
 Output [1]: [d_date_sk#127]
 
-(137) BroadcastHashJoin [codegen id : 6]
+(125) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [ws_sold_date_sk#126]
 Right keys [1]: [d_date_sk#127]
 Join type: Inner
 Join condition: None
 
-(138) Project [codegen id : 6]
+(126) Project [codegen id : 6]
 Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129]
 Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127]
 
-(139) Union
+(127) Union
 
-(140) HashAggregate [codegen id : 7]
+(128) HashAggregate [codegen id : 7]
 Input [2]: [quantity#116, list_price#117]
 Keys: []
 Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))]
 Aggregate Attributes [2]: [sum#130, count#131]
 Results [2]: [sum#132, count#133]
 
-(141) Exchange
+(129) Exchange
 Input [2]: [sum#132, count#133]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15]
 
-(142) HashAggregate [codegen id : 8]
+(130) HashAggregate [codegen id : 8]
 Input [2]: [sum#132, count#133]
 Keys: []
 Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))]
 Aggregate Attributes [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134]
 Results [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134 AS average_sales#135]
 
-Subquery:2 Hosting operator id = 124 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12
 
-Subquery:3 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12
+Subquery:3 Hosting operator id = 117 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12
 
-Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12
+Subquery:4 Hosting operator id = 122 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12
 
 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (147)
-+- * Project (146)
-   +- * Filter (145)
-      +- * ColumnarToRow (144)
-         +- Scan parquet spark_catalog.default.date_dim (143)
+BroadcastExchange (135)
++- * Project (134)
+   +- * Filter (133)
+      +- * ColumnarToRow (132)
+         +- Scan parquet spark_catalog.default.date_dim (131)
 
 
-(143) Scan parquet spark_catalog.default.date_dim
+(131) Scan parquet spark_catalog.default.date_dim
 Output [3]: [d_date_sk#36, d_year#136, d_moy#137]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
 
-(144) ColumnarToRow [codegen id : 1]
+(132) ColumnarToRow [codegen id : 1]
 Input [3]: [d_date_sk#36, d_year#136, d_moy#137]
 
-(145) Filter [codegen id : 1]
+(133) Filter [codegen id : 1]
 Input [3]: [d_date_sk#36, d_year#136, d_moy#137]
 Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 = 2001)) AND (d_moy#137 = 11)) AND isnotnull(d_date_sk#36))
 
-(146) Project [codegen id : 1]
+(134) Project [codegen id : 1]
 Output [1]: [d_date_sk#36]
 Input [3]: [d_date_sk#36, d_year#136, d_moy#137]
 
-(147) BroadcastExchange
+(135) BroadcastExchange
 Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
 
-Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (152)
-+- * Project (151)
-   +- * Filter (150)
-      +- * ColumnarToRow (149)
-         +- Scan parquet spark_catalog.default.date_dim (148)
+Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (140)
++- * Project (139)
+   +- * Filter (138)
+      +- * ColumnarToRow (137)
+         +- Scan parquet spark_catalog.default.date_dim (136)
 
 
-(148) Scan parquet spark_catalog.default.date_dim
+(136) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#13, d_year#138]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(149) ColumnarToRow [codegen id : 1]
+(137) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#138]
 
-(150) Filter [codegen id : 1]
+(138) Filter [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#138]
 Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#13))
 
-(151) Project [codegen id : 1]
+(139) Project [codegen id : 1]
 Output [1]: [d_date_sk#13]
 Input [2]: [d_date_sk#13, d_year#138]
 
-(152) BroadcastExchange
+(140) BroadcastExchange
 Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17]
 
-Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
 
-Subquery:8 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
 
-Subquery:9 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
+Subquery:9 Hosting operator id = 88 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
 
-Subquery:10 Hosting operator id = 80 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5
+Subquery:10 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5
 
-Subquery:11 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
+Subquery:11 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52]
 
-Subquery:12 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5
+Subquery:12 Hosting operator id = 90 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
index 460d252e030..7a7ce7f20b5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
@@ -1,21 +1,21 @@
 TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)]
-  WholeStageCodegen (134)
+  WholeStageCodegen (116)
     HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum]
       InputAdapter
         Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1
-          WholeStageCodegen (133)
+          WholeStageCodegen (115)
             HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
               Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id]
                 InputAdapter
                   Union
-                    WholeStageCodegen (44)
+                    WholeStageCodegen (38)
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           Subquery #3
                             WholeStageCodegen (8)
                               HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
                                 InputAdapter
-                                  Exchange #17
+                                  Exchange #15
                                     WholeStageCodegen (7)
                                       HashAggregate [quantity,list_price] [sum,count,sum,count]
                                         InputAdapter
@@ -28,7 +28,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                                                       Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
                                                         ReusedSubquery [d_date_sk] #2
                                                   InputAdapter
-                                                    ReusedExchange [d_date_sk] #9
+                                                    ReusedExchange [d_date_sk] #8
                                             WholeStageCodegen (4)
                                               Project [cs_quantity,cs_list_price]
                                                 BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -37,7 +37,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                                                       Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
                                                         ReusedSubquery [d_date_sk] #2
                                                   InputAdapter
-                                                    ReusedExchange [d_date_sk] #9
+                                                    ReusedExchange [d_date_sk] #8
                                             WholeStageCodegen (6)
                                               Project [ws_quantity,ws_list_price]
                                                 BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -46,223 +46,187 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                                                       Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
                                                         ReusedSubquery [d_date_sk] #2
                                                   InputAdapter
-                                                    ReusedExchange [d_date_sk] #9
+                                                    ReusedExchange [d_date_sk] #8
                           HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #2
-                                WholeStageCodegen (43)
+                                WholeStageCodegen (37)
                                   HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                                     Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
                                       BroadcastHashJoin [ss_item_sk,i_item_sk]
                                         Project [ss_item_sk,ss_quantity,ss_list_price]
                                           BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                            SortMergeJoin [ss_item_sk,ss_item_sk]
-                                              InputAdapter
-                                                WholeStageCodegen (2)
-                                                  Sort [ss_item_sk]
-                                                    InputAdapter
-                                                      Exchange [ss_item_sk] #3
-                                                        WholeStageCodegen (1)
-                                                          Filter [ss_item_sk]
-                                                            ColumnarToRow
-                                                              InputAdapter
-                                                                Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
-                                                                  SubqueryBroadcast [d_date_sk] #1
-                                                                    BroadcastExchange #4
-                                                                      WholeStageCodegen (1)
-                                                                        Project [d_date_sk]
-                                                                          Filter [d_year,d_moy,d_date_sk]
-                                                                            ColumnarToRow
-                                                                              InputAdapter
-                                                                                Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
-                                              InputAdapter
-                                                WholeStageCodegen (20)
-                                                  Sort [ss_item_sk]
-                                                    InputAdapter
-                                                      Exchange [ss_item_sk] #5
-                                                        WholeStageCodegen (19)
-                                                          Project [i_item_sk]
-                                                            BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
-                                                              Filter [i_brand_id,i_class_id,i_category_id]
+                                            BroadcastHashJoin [ss_item_sk,ss_item_sk]
+                                              Filter [ss_item_sk]
+                                                ColumnarToRow
+                                                  InputAdapter
+                                                    Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+                                                      SubqueryBroadcast [d_date_sk] #1
+                                                        BroadcastExchange #3
+                                                          WholeStageCodegen (1)
+                                                            Project [d_date_sk]
+                                                              Filter [d_year,d_moy,d_date_sk]
                                                                 ColumnarToRow
                                                                   InputAdapter
-                                                                    Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                                              InputAdapter
-                                                                BroadcastExchange #6
-                                                                  WholeStageCodegen (18)
-                                                                    SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                                                    Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+                                              InputAdapter
+                                                BroadcastExchange #4
+                                                  WholeStageCodegen (17)
+                                                    Project [i_item_sk]
+                                                      BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+                                                        Filter [i_brand_id,i_class_id,i_category_id]
+                                                          ColumnarToRow
+                                                            InputAdapter
+                                                              Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                        InputAdapter
+                                                          BroadcastExchange #5
+                                                            WholeStageCodegen (16)
+                                                              SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                                                InputAdapter
+                                                                  WholeStageCodegen (11)
+                                                                    Sort [brand_id,class_id,category_id]
                                                                       InputAdapter
-                                                                        WholeStageCodegen (13)
-                                                                          Sort [brand_id,class_id,category_id]
-                                                                            InputAdapter
-                                                                              Exchange [brand_id,class_id,category_id] #7
-                                                                                WholeStageCodegen (12)
-                                                                                  HashAggregate [brand_id,class_id,category_id]
-                                                                                    InputAdapter
-                                                                                      Exchange [brand_id,class_id,category_id] #8
-                                                                                        WholeStageCodegen (11)
-                                                                                          HashAggregate [brand_id,class_id,category_id]
-                                                                                            Project [i_brand_id,i_class_id,i_category_id]
-                                                                                              BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                                                                                Project [ss_item_sk]
-                                                                                                  BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                                    Filter [ss_item_sk]
-                                                                                                      ColumnarToRow
-                                                                                                        InputAdapter
-                                                                                                          Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
-                                                                                                            SubqueryBroadcast [d_date_sk] #2
-                                                                                                              BroadcastExchange #9
-                                                                                                                WholeStageCodegen (1)
-                                                                                                                  Project [d_date_sk]
-                                                                                                                    Filter [d_year,d_date_sk]
-                                                                                                                      ColumnarToRow
-                                                                                                                        InputAdapter
-                                                                                                                          Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
-                                                                                                    InputAdapter
-                                                                                                      ReusedExchange [d_date_sk] #9
-                                                                                                InputAdapter
-                                                                                                  BroadcastExchange #10
-                                                                                                    WholeStageCodegen (10)
-                                                                                                      SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                                        Exchange [brand_id,class_id,category_id] #6
+                                                                          WholeStageCodegen (10)
+                                                                            HashAggregate [brand_id,class_id,category_id]
+                                                                              InputAdapter
+                                                                                Exchange [brand_id,class_id,category_id] #7
+                                                                                  WholeStageCodegen (9)
+                                                                                    HashAggregate [brand_id,class_id,category_id]
+                                                                                      Project [i_brand_id,i_class_id,i_category_id]
+                                                                                        BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                                                                          Project [ss_item_sk]
+                                                                                            BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                                              Filter [ss_item_sk]
+                                                                                                ColumnarToRow
+                                                                                                  InputAdapter
+                                                                                                    Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+                                                                                                      SubqueryBroadcast [d_date_sk] #2
+                                                                                                        BroadcastExchange #8
+                                                                                                          WholeStageCodegen (1)
+                                                                                                            Project [d_date_sk]
+                                                                                                              Filter [d_year,d_date_sk]
+                                                                                                                ColumnarToRow
+                                                                                                                  InputAdapter
+                                                                                                                    Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+                                                                                              InputAdapter
+                                                                                                ReusedExchange [d_date_sk] #8
+                                                                                          InputAdapter
+                                                                                            BroadcastExchange #9
+                                                                                              WholeStageCodegen (8)
+                                                                                                SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                                                                  InputAdapter
+                                                                                                    WholeStageCodegen (3)
+                                                                                                      Sort [i_brand_id,i_class_id,i_category_id]
                                                                                                         InputAdapter
-                                                                                                          WholeStageCodegen (5)
-                                                                                                            Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                                              InputAdapter
-                                                                                                                Exchange [i_brand_id,i_class_id,i_category_id] #11
-                                                                                                                  WholeStageCodegen (4)
-                                                                                                                    Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                                                                                                      ColumnarToRow
-                                                                                                                        InputAdapter
-                                                                                                                          Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                          Exchange [i_brand_id,i_class_id,i_category_id] #10
+                                                                                                            WholeStageCodegen (2)
+                                                                                                              Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                                ColumnarToRow
+                                                                                                                  InputAdapter
+                                                                                                                    Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                  InputAdapter
+                                                                                                    WholeStageCodegen (7)
+                                                                                                      Sort [i_brand_id,i_class_id,i_category_id]
                                                                                                         InputAdapter
-                                                                                                          WholeStageCodegen (9)
-                                                                                                            Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                                              InputAdapter
-                                                                                                                Exchange [i_brand_id,i_class_id,i_category_id] #12
-                                                                                                                  WholeStageCodegen (8)
-                                                                                                                    Project [i_brand_id,i_class_id,i_category_id]
-                                                                                                                      BroadcastHashJoin [cs_item_sk,i_item_sk]
-                                                                                                                        Project [cs_item_sk]
-                                                                                                                          BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                                                                                            Filter [cs_item_sk]
-                                                                                                                              ColumnarToRow
-                                                                                                                                InputAdapter
-                                                                                                                                  Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
-                                                                                                                                    ReusedSubquery [d_date_sk] #2
+                                                                                                          Exchange [i_brand_id,i_class_id,i_category_id] #11
+                                                                                                            WholeStageCodegen (6)
+                                                                                                              Project [i_brand_id,i_class_id,i_category_id]
+                                                                                                                BroadcastHashJoin [cs_item_sk,i_item_sk]
+                                                                                                                  Project [cs_item_sk]
+                                                                                                                    BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+                                                                                                                      Filter [cs_item_sk]
+                                                                                                                        ColumnarToRow
+                                                                                                                          InputAdapter
+                                                                                                                            Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+                                                                                                                              ReusedSubquery [d_date_sk] #2
+                                                                                                                      InputAdapter
+                                                                                                                        ReusedExchange [d_date_sk] #8
+                                                                                                                  InputAdapter
+                                                                                                                    BroadcastExchange #12
+                                                                                                                      WholeStageCodegen (5)
+                                                                                                                        Filter [i_item_sk]
+                                                                                                                          ColumnarToRow
                                                                                                                             InputAdapter
-                                                                                                                              ReusedExchange [d_date_sk] #9
-                                                                                                                        InputAdapter
-                                                                                                                          BroadcastExchange #13
-                                                                                                                            WholeStageCodegen (7)
-                                                                                                                              Filter [i_item_sk]
-                                                                                                                                ColumnarToRow
-                                                                                                                                  InputAdapter
-                                                                                                                                    Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                                              Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                InputAdapter
+                                                                  WholeStageCodegen (15)
+                                                                    Sort [i_brand_id,i_class_id,i_category_id]
                                                                       InputAdapter
-                                                                        WholeStageCodegen (17)
-                                                                          Sort [i_brand_id,i_class_id,i_category_id]
-                                                                            InputAdapter
-                                                                              Exchange [i_brand_id,i_class_id,i_category_id] #14
-                                                                                WholeStageCodegen (16)
-                                                                                  Project [i_brand_id,i_class_id,i_category_id]
-                                                                                    BroadcastHashJoin [ws_item_sk,i_item_sk]
-                                                                                      Project [ws_item_sk]
-                                                                                        BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                                                                          Filter [ws_item_sk]
-                                                                                            ColumnarToRow
-                                                                                              InputAdapter
-                                                                                                Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
-                                                                                                  ReusedSubquery [d_date_sk] #2
-                                                                                          InputAdapter
-                                                                                            ReusedExchange [d_date_sk] #9
-                                                                                      InputAdapter
-                                                                                        ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
+                                                                        Exchange [i_brand_id,i_class_id,i_category_id] #13
+                                                                          WholeStageCodegen (14)
+                                                                            Project [i_brand_id,i_class_id,i_category_id]
+                                                                              BroadcastHashJoin [ws_item_sk,i_item_sk]
+                                                                                Project [ws_item_sk]
+                                                                                  BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+                                                                                    Filter [ws_item_sk]
+                                                                                      ColumnarToRow
+                                                                                        InputAdapter
+                                                                                          Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+                                                                                            ReusedSubquery [d_date_sk] #2
+                                                                                    InputAdapter
+                                                                                      ReusedExchange [d_date_sk] #8
+                                                                                InputAdapter
+                                                                                  ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12
                                             InputAdapter
-                                              ReusedExchange [d_date_sk] #4
+                                              ReusedExchange [d_date_sk] #3
                                         InputAdapter
-                                          BroadcastExchange #15
-                                            WholeStageCodegen (42)
-                                              SortMergeJoin [i_item_sk,ss_item_sk]
-                                                InputAdapter
-                                                  WholeStageCodegen (23)
-                                                    Sort [i_item_sk]
-                                                      InputAdapter
-                                                        Exchange [i_item_sk] #16
-                                                          WholeStageCodegen (22)
-                                                            Filter [i_item_sk]
-                                                              ColumnarToRow
-                                                                InputAdapter
-                                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                          BroadcastExchange #14
+                                            WholeStageCodegen (36)
+                                              BroadcastHashJoin [i_item_sk,ss_item_sk]
+                                                Filter [i_item_sk]
+                                                  ColumnarToRow
+                                                    InputAdapter
+                                                      Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
                                                 InputAdapter
-                                                  WholeStageCodegen (41)
-                                                    Sort [ss_item_sk]
-                                                      InputAdapter
-                                                        ReusedExchange [ss_item_sk] #5
-                    WholeStageCodegen (88)
+                                                  ReusedExchange [ss_item_sk] #4
+                    WholeStageCodegen (76)
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           ReusedSubquery [average_sales] #3
                           HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
-                              Exchange [i_brand_id,i_class_id,i_category_id] #18
-                                WholeStageCodegen (87)
+                              Exchange [i_brand_id,i_class_id,i_category_id] #16
+                                WholeStageCodegen (75)
                                   HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                                     Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id]
                                       BroadcastHashJoin [cs_item_sk,i_item_sk]
                                         Project [cs_item_sk,cs_quantity,cs_list_price]
                                           BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                            SortMergeJoin [cs_item_sk,ss_item_sk]
-                                              InputAdapter
-                                                WholeStageCodegen (46)
-                                                  Sort [cs_item_sk]
-                                                    InputAdapter
-                                                      Exchange [cs_item_sk] #19
-                                                        WholeStageCodegen (45)
-                                                          Filter [cs_item_sk]
-                                                            ColumnarToRow
-                                                              InputAdapter
-                                                                Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
-                                                                  ReusedSubquery [d_date_sk] #1
+                                            BroadcastHashJoin [cs_item_sk,ss_item_sk]
+                                              Filter [cs_item_sk]
+                                                ColumnarToRow
+                                                  InputAdapter
+                                                    Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+                                                      ReusedSubquery [d_date_sk] #1
                                               InputAdapter
-                                                WholeStageCodegen (64)
-                                                  Sort [ss_item_sk]
-                                                    InputAdapter
-                                                      ReusedExchange [ss_item_sk] #5
+                                                ReusedExchange [ss_item_sk] #4
                                             InputAdapter
-                                              ReusedExchange [d_date_sk] #4
+                                              ReusedExchange [d_date_sk] #3
                                         InputAdapter
-                                          ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
-                    WholeStageCodegen (132)
+                                          ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+                    WholeStageCodegen (114)
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           ReusedSubquery [average_sales] #3
                           HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
-                              Exchange [i_brand_id,i_class_id,i_category_id] #20
-                                WholeStageCodegen (131)
+                              Exchange [i_brand_id,i_class_id,i_category_id] #17
+                                WholeStageCodegen (113)
                                   HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                                     Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id]
                                       BroadcastHashJoin [ws_item_sk,i_item_sk]
                                         Project [ws_item_sk,ws_quantity,ws_list_price]
                                           BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                            SortMergeJoin [ws_item_sk,ss_item_sk]
-                                              InputAdapter
-                                                WholeStageCodegen (90)
-                                                  Sort [ws_item_sk]
-                                                    InputAdapter
-                                                      Exchange [ws_item_sk] #21
-                                                        WholeStageCodegen (89)
-                                                          Filter [ws_item_sk]
-                                                            ColumnarToRow
-                                                              InputAdapter
-                                                                Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
-                                                                  ReusedSubquery [d_date_sk] #1
+                                            BroadcastHashJoin [ws_item_sk,ss_item_sk]
+                                              Filter [ws_item_sk]
+                                                ColumnarToRow
+                                                  InputAdapter
+                                                    Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+                                                      ReusedSubquery [d_date_sk] #1
                                               InputAdapter
-                                                WholeStageCodegen (108)
-                                                  Sort [ss_item_sk]
-                                                    InputAdapter
-                                                      ReusedExchange [ss_item_sk] #5
+                                                ReusedExchange [ss_item_sk] #4
                                             InputAdapter
-                                              ReusedExchange [d_date_sk] #4
+                                              ReusedExchange [d_date_sk] #3
                                         InputAdapter
-                                          ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
+                                          ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
index 22b9222624d..99573a62028 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
@@ -1,103 +1,94 @@
 == Physical Plan ==
-TakeOrderedAndProject (99)
-+- * BroadcastHashJoin Inner BuildRight (98)
-   :- * Filter (78)
-   :  +- * HashAggregate (77)
-   :     +- Exchange (76)
-   :        +- * HashAggregate (75)
-   :           +- * Project (74)
-   :              +- * BroadcastHashJoin Inner BuildRight (73)
-   :                 :- * Project (63)
-   :                 :  +- * BroadcastHashJoin Inner BuildRight (62)
-   :                 :     :- * SortMergeJoin LeftSemi (60)
-   :                 :     :  :- * Sort (5)
-   :                 :     :  :  +- Exchange (4)
-   :                 :     :  :     +- * Filter (3)
-   :                 :     :  :        +- * ColumnarToRow (2)
-   :                 :     :  :           +- Scan parquet spark_catalog.default.store_sales (1)
-   :                 :     :  +- * Sort (59)
-   :                 :     :     +- Exchange (58)
-   :                 :     :        +- * Project (57)
-   :                 :     :           +- * BroadcastHashJoin Inner BuildRight (56)
-   :                 :     :              :- * Filter (8)
-   :                 :     :              :  +- * ColumnarToRow (7)
-   :                 :     :              :     +- Scan parquet spark_catalog.default.item (6)
-   :                 :     :              +- BroadcastExchange (55)
-   :                 :     :                 +- * SortMergeJoin LeftSemi (54)
-   :                 :     :                    :- * Sort (42)
-   :                 :     :                    :  +- Exchange (41)
-   :                 :     :                    :     +- * HashAggregate (40)
-   :                 :     :                    :        +- Exchange (39)
-   :                 :     :                    :           +- * HashAggregate (38)
-   :                 :     :                    :              +- * Project (37)
-   :                 :     :                    :                 +- * BroadcastHashJoin Inner BuildRight (36)
-   :                 :     :                    :                    :- * Project (14)
-   :                 :     :                    :                    :  +- * BroadcastHashJoin Inner BuildRight (13)
-   :                 :     :                    :                    :     :- * Filter (11)
-   :                 :     :                    :                    :     :  +- * ColumnarToRow (10)
-   :                 :     :                    :                    :     :     +- Scan parquet spark_catalog.default.store_sales (9)
-   :                 :     :                    :                    :     +- ReusedExchange (12)
-   :                 :     :                    :                    +- BroadcastExchange (35)
-   :                 :     :                    :                       +- * SortMergeJoin LeftSemi (34)
-   :                 :     :                    :                          :- * Sort (19)
-   :                 :     :                    :                          :  +- Exchange (18)
-   :                 :     :                    :                          :     +- * Filter (17)
-   :                 :     :                    :                          :        +- * ColumnarToRow (16)
-   :                 :     :                    :                          :           +- Scan parquet spark_catalog.default.item (15)
-   :                 :     :                    :                          +- * Sort (33)
-   :                 :     :                    :                             +- Exchange (32)
-   :                 :     :                    :                                +- * Project (31)
-   :                 :     :                    :                                   +- * BroadcastHashJoin Inner BuildRight (30)
-   :                 :     :                    :                                      :- * Project (25)
-   :                 :     :                    :                                      :  +- * BroadcastHashJoin Inner BuildRight (24)
-   :                 :     :                    :                                      :     :- * Filter (22)
-   :                 :     :                    :                                      :     :  +- * ColumnarToRow (21)
-   :                 :     :                    :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (20)
-   :                 :     :                    :                                      :     +- ReusedExchange (23)
-   :                 :     :                    :                                      +- BroadcastExchange (29)
-   :                 :     :                    :                                         +- * Filter (28)
-   :                 :     :                    :                                            +- * ColumnarToRow (27)
-   :                 :     :                    :                                               +- Scan parquet spark_catalog.default.item (26)
-   :                 :     :                    +- * Sort (53)
-   :                 :     :                       +- Exchange (52)
-   :                 :     :                          +- * Project (51)
-   :                 :     :                             +- * BroadcastHashJoin Inner BuildRight (50)
-   :                 :     :                                :- * Project (48)
-   :                 :     :                                :  +- * BroadcastHashJoin Inner BuildRight (47)
-   :                 :     :                                :     :- * Filter (45)
-   :                 :     :                                :     :  +- * ColumnarToRow (44)
-   :                 :     :                                :     :     +- Scan parquet spark_catalog.default.web_sales (43)
-   :                 :     :                                :     +- ReusedExchange (46)
-   :                 :     :                                +- ReusedExchange (49)
-   :                 :     +- ReusedExchange (61)
-   :                 +- BroadcastExchange (72)
-   :                    +- * SortMergeJoin LeftSemi (71)
-   :                       :- * Sort (68)
-   :                       :  +- Exchange (67)
-   :                       :     +- * Filter (66)
-   :                       :        +- * ColumnarToRow (65)
-   :                       :           +- Scan parquet spark_catalog.default.item (64)
-   :                       +- * Sort (70)
-   :                          +- ReusedExchange (69)
-   +- BroadcastExchange (97)
-      +- * Filter (96)
-         +- * HashAggregate (95)
-            +- Exchange (94)
-               +- * HashAggregate (93)
-                  +- * Project (92)
-                     +- * BroadcastHashJoin Inner BuildRight (91)
-                        :- * Project (89)
-                        :  +- * BroadcastHashJoin Inner BuildRight (88)
-                        :     :- * SortMergeJoin LeftSemi (86)
-                        :     :  :- * Sort (83)
-                        :     :  :  +- Exchange (82)
-                        :     :  :     +- * Filter (81)
-                        :     :  :        +- * ColumnarToRow (80)
-                        :     :  :           +- Scan parquet spark_catalog.default.store_sales (79)
-                        :     :  +- * Sort (85)
-                        :     :     +- ReusedExchange (84)
-                        :     +- ReusedExchange (87)
-                        +- ReusedExchange (90)
+TakeOrderedAndProject (90)
++- * BroadcastHashJoin Inner BuildRight (89)
+   :- * Filter (72)
+   :  +- * HashAggregate (71)
+   :     +- Exchange (70)
+   :        +- * HashAggregate (69)
+   :           +- * Project (68)
+   :              +- * BroadcastHashJoin Inner BuildRight (67)
+   :                 :- * Project (60)
+   :                 :  +- * BroadcastHashJoin Inner BuildRight (59)
+   :                 :     :- * BroadcastHashJoin LeftSemi BuildRight (57)
+   :                 :     :  :- * Filter (3)
+   :                 :     :  :  +- * ColumnarToRow (2)
+   :                 :     :  :     +- Scan parquet spark_catalog.default.store_sales (1)
+   :                 :     :  +- BroadcastExchange (56)
+   :                 :     :     +- * Project (55)
+   :                 :     :        +- * BroadcastHashJoin Inner BuildRight (54)
+   :                 :     :           :- * Filter (6)
+   :                 :     :           :  +- * ColumnarToRow (5)
+   :                 :     :           :     +- Scan parquet spark_catalog.default.item (4)
+   :                 :     :           +- BroadcastExchange (53)
+   :                 :     :              +- * SortMergeJoin LeftSemi (52)
+   :                 :     :                 :- * Sort (40)
+   :                 :     :                 :  +- Exchange (39)
+   :                 :     :                 :     +- * HashAggregate (38)
+   :                 :     :                 :        +- Exchange (37)
+   :                 :     :                 :           +- * HashAggregate (36)
+   :                 :     :                 :              +- * Project (35)
+   :                 :     :                 :                 +- * BroadcastHashJoin Inner BuildRight (34)
+   :                 :     :                 :                    :- * Project (12)
+   :                 :     :                 :                    :  +- * BroadcastHashJoin Inner BuildRight (11)
+   :                 :     :                 :                    :     :- * Filter (9)
+   :                 :     :                 :                    :     :  +- * ColumnarToRow (8)
+   :                 :     :                 :                    :     :     +- Scan parquet spark_catalog.default.store_sales (7)
+   :                 :     :                 :                    :     +- ReusedExchange (10)
+   :                 :     :                 :                    +- BroadcastExchange (33)
+   :                 :     :                 :                       +- * SortMergeJoin LeftSemi (32)
+   :                 :     :                 :                          :- * Sort (17)
+   :                 :     :                 :                          :  +- Exchange (16)
+   :                 :     :                 :                          :     +- * Filter (15)
+   :                 :     :                 :                          :        +- * ColumnarToRow (14)
+   :                 :     :                 :                          :           +- Scan parquet spark_catalog.default.item (13)
+   :                 :     :                 :                          +- * Sort (31)
+   :                 :     :                 :                             +- Exchange (30)
+   :                 :     :                 :                                +- * Project (29)
+   :                 :     :                 :                                   +- * BroadcastHashJoin Inner BuildRight (28)
+   :                 :     :                 :                                      :- * Project (23)
+   :                 :     :                 :                                      :  +- * BroadcastHashJoin Inner BuildRight (22)
+   :                 :     :                 :                                      :     :- * Filter (20)
+   :                 :     :                 :                                      :     :  +- * ColumnarToRow (19)
+   :                 :     :                 :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (18)
+   :                 :     :                 :                                      :     +- ReusedExchange (21)
+   :                 :     :                 :                                      +- BroadcastExchange (27)
+   :                 :     :                 :                                         +- * Filter (26)
+   :                 :     :                 :                                            +- * ColumnarToRow (25)
+   :                 :     :                 :                                               +- Scan parquet spark_catalog.default.item (24)
+   :                 :     :                 +- * Sort (51)
+   :                 :     :                    +- Exchange (50)
+   :                 :     :                       +- * Project (49)
+   :                 :     :                          +- * BroadcastHashJoin Inner BuildRight (48)
+   :                 :     :                             :- * Project (46)
+   :                 :     :                             :  +- * BroadcastHashJoin Inner BuildRight (45)
+   :                 :     :                             :     :- * Filter (43)
+   :                 :     :                             :     :  +- * ColumnarToRow (42)
+   :                 :     :                             :     :     +- Scan parquet spark_catalog.default.web_sales (41)
+   :                 :     :                             :     +- ReusedExchange (44)
+   :                 :     :                             +- ReusedExchange (47)
+   :                 :     +- ReusedExchange (58)
+   :                 +- BroadcastExchange (66)
+   :                    +- * BroadcastHashJoin LeftSemi BuildRight (65)
+   :                       :- * Filter (63)
+   :                       :  +- * ColumnarToRow (62)
+   :                       :     +- Scan parquet spark_catalog.default.item (61)
+   :                       +- ReusedExchange (64)
+   +- BroadcastExchange (88)
+      +- * Filter (87)
+         +- * HashAggregate (86)
+            +- Exchange (85)
+               +- * HashAggregate (84)
+                  +- * Project (83)
+                     +- * BroadcastHashJoin Inner BuildRight (82)
+                        :- * Project (80)
+                        :  +- * BroadcastHashJoin Inner BuildRight (79)
+                        :     :- * BroadcastHashJoin LeftSemi BuildRight (77)
+                        :     :  :- * Filter (75)
+                        :     :  :  +- * ColumnarToRow (74)
+                        :     :  :     +- Scan parquet spark_catalog.default.store_sales (73)
+                        :     :  +- ReusedExchange (76)
+                        :     +- ReusedExchange (78)
+                        +- ReusedExchange (81)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -108,36 +99,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 Condition : isnotnull(ss_item_sk#1)
 
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
 
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
 Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Batched: true
 Location: InMemoryFileIndex []
@@ -145,49 +128,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int>
 
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Condition : isnotnull(ss_item_sk#10)
 
-(12) ReusedExchange [Reuses operator id: 132]
+(10) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#13]
 
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_sold_date_sk#11]
 Right keys [1]: [d_date_sk#13]
 Join type: Inner
 Join condition: None
 
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
 Output [1]: [ss_item_sk#10]
 Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
 
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
 
-(18) Exchange
+(16) Exchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
 
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
 
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
 Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Batched: true
 Location: InMemoryFileIndex []
@@ -195,109 +178,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int>
 
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Condition : isnotnull(cs_item_sk#18)
 
-(23) ReusedExchange [Reuses operator id: 132]
+(21) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#20]
 
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_sold_date_sk#19]
 Right keys [1]: [d_date_sk#20]
 Join type: Inner
 Join condition: None
 
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
 Output [1]: [cs_item_sk#18]
 Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
 
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Condition : isnotnull(i_item_sk#21)
 
-(29) BroadcastExchange
+(27) BroadcastExchange
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
 
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_item_sk#18]
 Right keys [1]: [i_item_sk#21]
 Join type: Inner
 Join condition: None
 
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
 Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(32) Exchange
+(30) Exchange
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
 
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
 
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
 Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
 Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
 Join type: LeftSemi
 Join condition: None
 
-(35) BroadcastExchange
+(33) BroadcastExchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_item_sk#10]
 Right keys [1]: [i_item_sk#14]
 Join type: Inner
 Join condition: None
 
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
 Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
 Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(39) Exchange
+(37) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(41) Exchange
+(39) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
 
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
 Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Batched: true
 Location: InMemoryFileIndex []
@@ -305,166 +288,150 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int>
 
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Condition : isnotnull(ws_item_sk#28)
 
-(46) ReusedExchange [Reuses operator id: 132]
+(44) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#30]
 
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_sold_date_sk#29]
 Right keys [1]: [d_date_sk#30]
 Join type: Inner
 Join condition: None
 
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
 Output [1]: [ws_item_sk#28]
 Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
 
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
 Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_item_sk#28]
 Right keys [1]: [i_item_sk#31]
 Join type: Inner
 Join condition: None
 
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
 Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(52) Exchange
+(50) Exchange
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
 
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
 
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
 Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
 Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
 Join type: LeftSemi
 Join condition: None
 
-(55) BroadcastExchange
+(53) BroadcastExchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
 
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
 Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
 Right keys [3]: [brand_id#25, class_id#26, category_id#27]
 Join type: Inner
 Join condition: None
 
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
 Output [1]: [i_item_sk#6 AS ss_item_sk#35]
 Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
 
-(58) Exchange
+(56) BroadcastExchange
 Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
 
-(59) Sort [codegen id : 20]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(61) ReusedExchange [Reuses operator id: 123]
+(58) ReusedExchange [Reuses operator id: 114]
 Output [1]: [d_date_sk#36]
 
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_sold_date_sk#4]
 Right keys [1]: [d_date_sk#36]
 Join type: Inner
 Join condition: None
 
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
 Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
 Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
 
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40))
 
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
 Left keys [1]: [i_item_sk#37]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(72) BroadcastExchange
+(66) BroadcastExchange
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
 
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [i_item_sk#37]
 Join type: Inner
 Join condition: None
 
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
 Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
-(76) Exchange
+(70) Exchange
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
-(77) HashAggregate [codegen id : 88]
+(71) HashAggregate [codegen id : 76]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
 Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
-(78) Filter [codegen id : 88]
+(72) Filter [codegen id : 76]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
 Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(79) Scan parquet spark_catalog.default.store_sales
+(73) Scan parquet spark_catalog.default.store_sales
 Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
 Batched: true
 Location: InMemoryFileIndex []
@@ -472,358 +439,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(80) ColumnarToRow [codegen id : 44]
+(74) ColumnarToRow [codegen id : 74]
 Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
 
-(81) Filter [codegen id : 44]
+(75) Filter [codegen id : 74]
 Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
 Condition : isnotnull(ss_item_sk#54)
 
-(82) Exchange
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: hashpartitioning(ss_item_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(83) Sort [codegen id : 45]
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0
-
-(84) ReusedExchange [Reuses operator id: 58]
+(76) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(85) Sort [codegen id : 63]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(86) SortMergeJoin [codegen id : 86]
+(77) BroadcastHashJoin [codegen id : 74]
 Left keys [1]: [ss_item_sk#54]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(87) ReusedExchange [Reuses operator id: 137]
+(78) ReusedExchange [Reuses operator id: 128]
 Output [1]: [d_date_sk#59]
 
-(88) BroadcastHashJoin [codegen id : 86]
+(79) BroadcastHashJoin [codegen id : 74]
 Left keys [1]: [ss_sold_date_sk#57]
 Right keys [1]: [d_date_sk#59]
 Join type: Inner
 Join condition: None
 
-(89) Project [codegen id : 86]
+(80) Project [codegen id : 74]
 Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56]
 Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59]
 
-(90) ReusedExchange [Reuses operator id: 72]
+(81) ReusedExchange [Reuses operator id: 66]
 Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
 
-(91) BroadcastHashJoin [codegen id : 86]
+(82) BroadcastHashJoin [codegen id : 74]
 Left keys [1]: [ss_item_sk#54]
 Right keys [1]: [i_item_sk#60]
 Join type: Inner
 Join condition: None
 
-(92) Project [codegen id : 86]
+(83) Project [codegen id : 74]
 Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
 Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
 
-(93) HashAggregate [codegen id : 86]
+(84) HashAggregate [codegen id : 74]
 Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
 Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
 Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 
-(94) Exchange
+(85) Exchange
 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
-Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
-(95) HashAggregate [codegen id : 87]
+(86) HashAggregate [codegen id : 75]
 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
 Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
 Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
 
-(96) Filter [codegen id : 87]
+(87) Filter [codegen id : 75]
 Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
 Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(97) BroadcastExchange
+(88) BroadcastExchange
 Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
-Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=16]
+Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13]
 
-(98) BroadcastHashJoin [codegen id : 88]
+(89) BroadcastHashJoin [codegen id : 76]
 Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
 Join type: Inner
 Join condition: None
 
-(99) TakeOrderedAndProject
+(90) TakeOrderedAndProject
 Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
 Arguments: 100, [i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
 
 ===== Subqueries =====
 
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
-* HashAggregate (118)
-+- Exchange (117)
-   +- * HashAggregate (116)
-      +- Union (115)
-         :- * Project (104)
-         :  +- * BroadcastHashJoin Inner BuildRight (103)
-         :     :- * ColumnarToRow (101)
-         :     :  +- Scan parquet spark_catalog.default.store_sales (100)
-         :     +- ReusedExchange (102)
-         :- * Project (109)
-         :  +- * BroadcastHashJoin Inner BuildRight (108)
-         :     :- * ColumnarToRow (106)
-         :     :  +- Scan parquet spark_catalog.default.catalog_sales (105)
-         :     +- ReusedExchange (107)
-         +- * Project (114)
-            +- * BroadcastHashJoin Inner BuildRight (113)
-               :- * ColumnarToRow (111)
-               :  +- Scan parquet spark_catalog.default.web_sales (110)
-               +- ReusedExchange (112)
-
-
-(100) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
+* HashAggregate (109)
++- Exchange (108)
+   +- * HashAggregate (107)
+      +- Union (106)
+         :- * Project (95)
+         :  +- * BroadcastHashJoin Inner BuildRight (94)
+         :     :- * ColumnarToRow (92)
+         :     :  +- Scan parquet spark_catalog.default.store_sales (91)
+         :     +- ReusedExchange (93)
+         :- * Project (100)
+         :  +- * BroadcastHashJoin Inner BuildRight (99)
+         :     :- * ColumnarToRow (97)
+         :     :  +- Scan parquet spark_catalog.default.catalog_sales (96)
+         :     +- ReusedExchange (98)
+         +- * Project (105)
+            +- * BroadcastHashJoin Inner BuildRight (104)
+               :- * ColumnarToRow (102)
+               :  +- Scan parquet spark_catalog.default.web_sales (101)
+               +- ReusedExchange (103)
+
+
+(91) Scan parquet spark_catalog.default.store_sales
 Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)]
 ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(101) ColumnarToRow [codegen id : 2]
+(92) ColumnarToRow [codegen id : 2]
 Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
 
-(102) ReusedExchange [Reuses operator id: 132]
+(93) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#78]
 
-(103) BroadcastHashJoin [codegen id : 2]
+(94) BroadcastHashJoin [codegen id : 2]
 Left keys [1]: [ss_sold_date_sk#77]
 Right keys [1]: [d_date_sk#78]
 Join type: Inner
 Join condition: None
 
-(104) Project [codegen id : 2]
+(95) Project [codegen id : 2]
 Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80]
 Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78]
 
-(105) Scan parquet spark_catalog.default.catalog_sales
+(96) Scan parquet spark_catalog.default.catalog_sales
 Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)]
 ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
 
-(106) ColumnarToRow [codegen id : 4]
+(97) ColumnarToRow [codegen id : 4]
 Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
 
-(107) ReusedExchange [Reuses operator id: 132]
+(98) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#84]
 
-(108) BroadcastHashJoin [codegen id : 4]
+(99) BroadcastHashJoin [codegen id : 4]
 Left keys [1]: [cs_sold_date_sk#83]
 Right keys [1]: [d_date_sk#84]
 Join type: Inner
 Join condition: None
 
-(109) Project [codegen id : 4]
+(100) Project [codegen id : 4]
 Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86]
 Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84]
 
-(110) Scan parquet spark_catalog.default.web_sales
+(101) Scan parquet spark_catalog.default.web_sales
 Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)]
 ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
 
-(111) ColumnarToRow [codegen id : 6]
+(102) ColumnarToRow [codegen id : 6]
 Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
 
-(112) ReusedExchange [Reuses operator id: 132]
+(103) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#90]
 
-(113) BroadcastHashJoin [codegen id : 6]
+(104) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [ws_sold_date_sk#89]
 Right keys [1]: [d_date_sk#90]
 Join type: Inner
 Join condition: None
 
-(114) Project [codegen id : 6]
+(105) Project [codegen id : 6]
 Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92]
 Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
 
-(115) Union
+(106) Union
 
-(116) HashAggregate [codegen id : 7]
+(107) HashAggregate [codegen id : 7]
 Input [2]: [quantity#79, list_price#80]
 Keys: []
 Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [2]: [sum#93, count#94]
 Results [2]: [sum#95, count#96]
 
-(117) Exchange
+(108) Exchange
 Input [2]: [sum#95, count#96]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14]
 
-(118) HashAggregate [codegen id : 8]
+(109) HashAggregate [codegen id : 8]
 Input [2]: [sum#95, count#96]
 Keys: []
 Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
 Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
 
-Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
 
-Subquery:3 Hosting operator id = 105 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
+Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
 
-Subquery:4 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
+Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
 
 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (123)
-+- * Project (122)
-   +- * Filter (121)
-      +- * ColumnarToRow (120)
-         +- Scan parquet spark_catalog.default.date_dim (119)
+BroadcastExchange (114)
++- * Project (113)
+   +- * Filter (112)
+      +- * ColumnarToRow (111)
+         +- Scan parquet spark_catalog.default.date_dim (110)
 
 
-(119) Scan parquet spark_catalog.default.date_dim
+(110) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#36, d_week_seq#99]
 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>
 
-(120) ColumnarToRow [codegen id : 1]
+(111) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#36, d_week_seq#99]
 
-(121) Filter [codegen id : 1]
+(112) Filter [codegen id : 1]
 Input [2]: [d_date_sk#36, d_week_seq#99]
 Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36))
 
-(122) Project [codegen id : 1]
+(113) Project [codegen id : 1]
 Output [1]: [d_date_sk#36]
 Input [2]: [d_date_sk#36, d_week_seq#99]
 
-(123) BroadcastExchange
+(114) BroadcastExchange
 Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15]
 
-Subquery:6 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
-* Project (127)
-+- * Filter (126)
-   +- * ColumnarToRow (125)
-      +- Scan parquet spark_catalog.default.date_dim (124)
+Subquery:6 Hosting operator id = 112 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
+* Project (118)
++- * Filter (117)
+   +- * ColumnarToRow (116)
+      +- Scan parquet spark_catalog.default.date_dim (115)
 
 
-(124) Scan parquet spark_catalog.default.date_dim
+(115) Scan parquet spark_catalog.default.date_dim
 Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)]
 ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
 
-(125) ColumnarToRow [codegen id : 1]
+(116) ColumnarToRow [codegen id : 1]
 Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 
-(126) Filter [codegen id : 1]
+(117) Filter [codegen id : 1]
 Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11))
 
-(127) Project [codegen id : 1]
+(118) Project [codegen id : 1]
 Output [1]: [d_week_seq#102]
 Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 
-Subquery:7 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (132)
-+- * Project (131)
-   +- * Filter (130)
-      +- * ColumnarToRow (129)
-         +- Scan parquet spark_catalog.default.date_dim (128)
+Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (123)
++- * Project (122)
+   +- * Filter (121)
+      +- * ColumnarToRow (120)
+         +- Scan parquet spark_catalog.default.date_dim (119)
 
 
-(128) Scan parquet spark_catalog.default.date_dim
+(119) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#13, d_year#106]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(129) ColumnarToRow [codegen id : 1]
+(120) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#106]
 
-(130) Filter [codegen id : 1]
+(121) Filter [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#106]
 Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1999)) AND (d_year#106 <= 2001)) AND isnotnull(d_date_sk#13))
 
-(131) Project [codegen id : 1]
+(122) Project [codegen id : 1]
 Output [1]: [d_date_sk#13]
 Input [2]: [d_date_sk#13, d_year#106]
 
-(132) BroadcastExchange
+(123) BroadcastExchange
 Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
 
-Subquery:8 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
 
-Subquery:9 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
 
-Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
 
-Subquery:11 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
-BroadcastExchange (137)
-+- * Project (136)
-   +- * Filter (135)
-      +- * ColumnarToRow (134)
-         +- Scan parquet spark_catalog.default.date_dim (133)
+Subquery:11 Hosting operator id = 73 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
+BroadcastExchange (128)
++- * Project (127)
+   +- * Filter (126)
+      +- * ColumnarToRow (125)
+         +- Scan parquet spark_catalog.default.date_dim (124)
 
 
-(133) Scan parquet spark_catalog.default.date_dim
+(124) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#59, d_week_seq#107]
 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>
 
-(134) ColumnarToRow [codegen id : 1]
+(125) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#59, d_week_seq#107]
 
-(135) Filter [codegen id : 1]
+(126) Filter [codegen id : 1]
 Input [2]: [d_date_sk#59, d_week_seq#107]
 Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59))
 
-(136) Project [codegen id : 1]
+(127) Project [codegen id : 1]
 Output [1]: [d_date_sk#59]
 Input [2]: [d_date_sk#59, d_week_seq#107]
 
-(137) BroadcastExchange
+(128) BroadcastExchange
 Input [1]: [d_date_sk#59]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17]
 
-Subquery:12 Hosting operator id = 135 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
-* Project (141)
-+- * Filter (140)
-   +- * ColumnarToRow (139)
-      +- Scan parquet spark_catalog.default.date_dim (138)
+Subquery:12 Hosting operator id = 126 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
+* Project (132)
++- * Filter (131)
+   +- * ColumnarToRow (130)
+      +- Scan parquet spark_catalog.default.date_dim (129)
 
 
-(138) Scan parquet spark_catalog.default.date_dim
+(129) Scan parquet spark_catalog.default.date_dim
 Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)]
 ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
 
-(139) ColumnarToRow [codegen id : 1]
+(130) ColumnarToRow [codegen id : 1]
 Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 
-(140) Filter [codegen id : 1]
+(131) Filter [codegen id : 1]
 Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1999)) AND (d_moy#112 = 12)) AND (d_dom#113 = 11))
 
-(141) Project [codegen id : 1]
+(132) Project [codegen id : 1]
 Output [1]: [d_week_seq#110]
 Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
index 8c67fff5b14..edd34864986 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
-  WholeStageCodegen (88)
+  WholeStageCodegen (76)
     BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
       Filter [sales]
         Subquery #4
           WholeStageCodegen (8)
             HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
               InputAdapter
-                Exchange #16
+                Exchange #14
                   WholeStageCodegen (7)
                     HashAggregate [quantity,list_price] [sum,count,sum,count]
                       InputAdapter
@@ -19,7 +19,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                     Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #8
+                                  ReusedExchange [d_date_sk] #7
                           WholeStageCodegen (4)
                             Project [cs_quantity,cs_list_price]
                               BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -28,7 +28,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                     Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #8
+                                  ReusedExchange [d_date_sk] #7
                           WholeStageCodegen (6)
                             Project [ws_quantity,ws_list_price]
                               BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -37,211 +37,184 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                     Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #8
+                                  ReusedExchange [d_date_sk] #7
         HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
           InputAdapter
             Exchange [i_brand_id,i_class_id,i_category_id] #1
-              WholeStageCodegen (43)
+              WholeStageCodegen (37)
                 HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                   Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
                     BroadcastHashJoin [ss_item_sk,i_item_sk]
                       Project [ss_item_sk,ss_quantity,ss_list_price]
                         BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                          SortMergeJoin [ss_item_sk,ss_item_sk]
-                            InputAdapter
-                              WholeStageCodegen (2)
-                                Sort [ss_item_sk]
-                                  InputAdapter
-                                    Exchange [ss_item_sk] #2
-                                      WholeStageCodegen (1)
-                                        Filter [ss_item_sk]
-                                          ColumnarToRow
-                                            InputAdapter
-                                              Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
-                                                SubqueryBroadcast [d_date_sk] #1
-                                                  BroadcastExchange #3
-                                                    WholeStageCodegen (1)
-                                                      Project [d_date_sk]
-                                                        Filter [d_week_seq,d_date_sk]
-                                                          Subquery #2
-                                                            WholeStageCodegen (1)
-                                                              Project [d_week_seq]
-                                                                Filter [d_year,d_moy,d_dom]
-                                                                  ColumnarToRow
-                                                                    InputAdapter
-                                                                      Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
-                                                          ColumnarToRow
-                                                            InputAdapter
-                                                              Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
-                            InputAdapter
-                              WholeStageCodegen (20)
-                                Sort [ss_item_sk]
-                                  InputAdapter
-                                    Exchange [ss_item_sk] #4
-                                      WholeStageCodegen (19)
-                                        Project [i_item_sk]
-                                          BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
-                                            Filter [i_brand_id,i_class_id,i_category_id]
+                          BroadcastHashJoin [ss_item_sk,ss_item_sk]
+                            Filter [ss_item_sk]
+                              ColumnarToRow
+                                InputAdapter
+                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+                                    SubqueryBroadcast [d_date_sk] #1
+                                      BroadcastExchange #2
+                                        WholeStageCodegen (1)
+                                          Project [d_date_sk]
+                                            Filter [d_week_seq,d_date_sk]
+                                              Subquery #2
+                                                WholeStageCodegen (1)
+                                                  Project [d_week_seq]
+                                                    Filter [d_year,d_moy,d_dom]
+                                                      ColumnarToRow
+                                                        InputAdapter
+                                                          Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
                                               ColumnarToRow
                                                 InputAdapter
-                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                            InputAdapter
-                                              BroadcastExchange #5
-                                                WholeStageCodegen (18)
-                                                  SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                                  Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+                            InputAdapter
+                              BroadcastExchange #3
+                                WholeStageCodegen (17)
+                                  Project [i_item_sk]
+                                    BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+                                      Filter [i_brand_id,i_class_id,i_category_id]
+                                        ColumnarToRow
+                                          InputAdapter
+                                            Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                      InputAdapter
+                                        BroadcastExchange #4
+                                          WholeStageCodegen (16)
+                                            SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                              InputAdapter
+                                                WholeStageCodegen (11)
+                                                  Sort [brand_id,class_id,category_id]
                                                     InputAdapter
-                                                      WholeStageCodegen (13)
-                                                        Sort [brand_id,class_id,category_id]
-                                                          InputAdapter
-                                                            Exchange [brand_id,class_id,category_id] #6
-                                                              WholeStageCodegen (12)
-                                                                HashAggregate [brand_id,class_id,category_id]
-                                                                  InputAdapter
-                                                                    Exchange [brand_id,class_id,category_id] #7
-                                                                      WholeStageCodegen (11)
-                                                                        HashAggregate [brand_id,class_id,category_id]
-                                                                          Project [i_brand_id,i_class_id,i_category_id]
-                                                                            BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                                                              Project [ss_item_sk]
-                                                                                BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                  Filter [ss_item_sk]
-                                                                                    ColumnarToRow
-                                                                                      InputAdapter
-                                                                                        Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
-                                                                                          SubqueryBroadcast [d_date_sk] #3
-                                                                                            BroadcastExchange #8
-                                                                                              WholeStageCodegen (1)
-                                                                                                Project [d_date_sk]
-                                                                                                  Filter [d_year,d_date_sk]
-                                                                                                    ColumnarToRow
-                                                                                                      InputAdapter
-                                                                                                        Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
-                                                                                  InputAdapter
-                                                                                    ReusedExchange [d_date_sk] #8
-                                                                              InputAdapter
-                                                                                BroadcastExchange #9
-                                                                                  WholeStageCodegen (10)
-                                                                                    SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                      Exchange [brand_id,class_id,category_id] #5
+                                                        WholeStageCodegen (10)
+                                                          HashAggregate [brand_id,class_id,category_id]
+                                                            InputAdapter
+                                                              Exchange [brand_id,class_id,category_id] #6
+                                                                WholeStageCodegen (9)
+                                                                  HashAggregate [brand_id,class_id,category_id]
+                                                                    Project [i_brand_id,i_class_id,i_category_id]
+                                                                      BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                                                        Project [ss_item_sk]
+                                                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                            Filter [ss_item_sk]
+                                                                              ColumnarToRow
+                                                                                InputAdapter
+                                                                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+                                                                                    SubqueryBroadcast [d_date_sk] #3
+                                                                                      BroadcastExchange #7
+                                                                                        WholeStageCodegen (1)
+                                                                                          Project [d_date_sk]
+                                                                                            Filter [d_year,d_date_sk]
+                                                                                              ColumnarToRow
+                                                                                                InputAdapter
+                                                                                                  Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+                                                                            InputAdapter
+                                                                              ReusedExchange [d_date_sk] #7
+                                                                        InputAdapter
+                                                                          BroadcastExchange #8
+                                                                            WholeStageCodegen (8)
+                                                                              SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                                                InputAdapter
+                                                                                  WholeStageCodegen (3)
+                                                                                    Sort [i_brand_id,i_class_id,i_category_id]
                                                                                       InputAdapter
-                                                                                        WholeStageCodegen (5)
-                                                                                          Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                            InputAdapter
-                                                                                              Exchange [i_brand_id,i_class_id,i_category_id] #10
-                                                                                                WholeStageCodegen (4)
-                                                                                                  Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                                                                                    ColumnarToRow
-                                                                                                      InputAdapter
-                                                                                                        Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                        Exchange [i_brand_id,i_class_id,i_category_id] #9
+                                                                                          WholeStageCodegen (2)
+                                                                                            Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                              ColumnarToRow
+                                                                                                InputAdapter
+                                                                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                InputAdapter
+                                                                                  WholeStageCodegen (7)
+                                                                                    Sort [i_brand_id,i_class_id,i_category_id]
                                                                                       InputAdapter
-                                                                                        WholeStageCodegen (9)
-                                                                                          Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                            InputAdapter
-                                                                                              Exchange [i_brand_id,i_class_id,i_category_id] #11
-                                                                                                WholeStageCodegen (8)
-                                                                                                  Project [i_brand_id,i_class_id,i_category_id]
-                                                                                                    BroadcastHashJoin [cs_item_sk,i_item_sk]
-                                                                                                      Project [cs_item_sk]
-                                                                                                        BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                                                                          Filter [cs_item_sk]
-                                                                                                            ColumnarToRow
-                                                                                                              InputAdapter
-                                                                                                                Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
-                                                                                                                  ReusedSubquery [d_date_sk] #3
+                                                                                        Exchange [i_brand_id,i_class_id,i_category_id] #10
+                                                                                          WholeStageCodegen (6)
+                                                                                            Project [i_brand_id,i_class_id,i_category_id]
+                                                                                              BroadcastHashJoin [cs_item_sk,i_item_sk]
+                                                                                                Project [cs_item_sk]
+                                                                                                  BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+                                                                                                    Filter [cs_item_sk]
+                                                                                                      ColumnarToRow
+                                                                                                        InputAdapter
+                                                                                                          Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+                                                                                                            ReusedSubquery [d_date_sk] #3
+                                                                                                    InputAdapter
+                                                                                                      ReusedExchange [d_date_sk] #7
+                                                                                                InputAdapter
+                                                                                                  BroadcastExchange #11
+                                                                                                    WholeStageCodegen (5)
+                                                                                                      Filter [i_item_sk]
+                                                                                                        ColumnarToRow
                                                                                                           InputAdapter
-                                                                                                            ReusedExchange [d_date_sk] #8
-                                                                                                      InputAdapter
-                                                                                                        BroadcastExchange #12
-                                                                                                          WholeStageCodegen (7)
-                                                                                                            Filter [i_item_sk]
-                                                                                                              ColumnarToRow
-                                                                                                                InputAdapter
-                                                                                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                            Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                              InputAdapter
+                                                WholeStageCodegen (15)
+                                                  Sort [i_brand_id,i_class_id,i_category_id]
                                                     InputAdapter
-                                                      WholeStageCodegen (17)
-                                                        Sort [i_brand_id,i_class_id,i_category_id]
-                                                          InputAdapter
-                                                            Exchange [i_brand_id,i_class_id,i_category_id] #13
-                                                              WholeStageCodegen (16)
-                                                                Project [i_brand_id,i_class_id,i_category_id]
-                                                                  BroadcastHashJoin [ws_item_sk,i_item_sk]
-                                                                    Project [ws_item_sk]
-                                                                      BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                                                        Filter [ws_item_sk]
-                                                                          ColumnarToRow
-                                                                            InputAdapter
-                                                                              Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
-                                                                                ReusedSubquery [d_date_sk] #3
-                                                                        InputAdapter
-                                                                          ReusedExchange [d_date_sk] #8
-                                                                    InputAdapter
-                                                                      ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12
+                                                      Exchange [i_brand_id,i_class_id,i_category_id] #12
+                                                        WholeStageCodegen (14)
+                                                          Project [i_brand_id,i_class_id,i_category_id]
+                                                            BroadcastHashJoin [ws_item_sk,i_item_sk]
+                                                              Project [ws_item_sk]
+                                                                BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+                                                                  Filter [ws_item_sk]
+                                                                    ColumnarToRow
+                                                                      InputAdapter
+                                                                        Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+                                                                          ReusedSubquery [d_date_sk] #3
+                                                                  InputAdapter
+                                                                    ReusedExchange [d_date_sk] #7
+                                                              InputAdapter
+                                                                ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
                           InputAdapter
-                            ReusedExchange [d_date_sk] #3
+                            ReusedExchange [d_date_sk] #2
                       InputAdapter
-                        BroadcastExchange #14
-                          WholeStageCodegen (42)
-                            SortMergeJoin [i_item_sk,ss_item_sk]
-                              InputAdapter
-                                WholeStageCodegen (23)
-                                  Sort [i_item_sk]
-                                    InputAdapter
-                                      Exchange [i_item_sk] #15
-                                        WholeStageCodegen (22)
-                                          Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                            ColumnarToRow
-                                              InputAdapter
-                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                        BroadcastExchange #13
+                          WholeStageCodegen (36)
+                            BroadcastHashJoin [i_item_sk,ss_item_sk]
+                              Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                ColumnarToRow
+                                  InputAdapter
+                                    Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
                               InputAdapter
-                                WholeStageCodegen (41)
-                                  Sort [ss_item_sk]
-                                    InputAdapter
-                                      ReusedExchange [ss_item_sk] #4
+                                ReusedExchange [ss_item_sk] #3
       InputAdapter
-        BroadcastExchange #17
-          WholeStageCodegen (87)
+        BroadcastExchange #15
+          WholeStageCodegen (75)
             Filter [sales]
               ReusedSubquery [average_sales] #4
               HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                 InputAdapter
-                  Exchange [i_brand_id,i_class_id,i_category_id] #18
-                    WholeStageCodegen (86)
+                  Exchange [i_brand_id,i_class_id,i_category_id] #16
+                    WholeStageCodegen (74)
                       HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                         Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
                           BroadcastHashJoin [ss_item_sk,i_item_sk]
                             Project [ss_item_sk,ss_quantity,ss_list_price]
                               BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                SortMergeJoin [ss_item_sk,ss_item_sk]
-                                  InputAdapter
-                                    WholeStageCodegen (45)
-                                      Sort [ss_item_sk]
-                                        InputAdapter
-                                          Exchange [ss_item_sk] #19
-                                            WholeStageCodegen (44)
-                                              Filter [ss_item_sk]
-                                                ColumnarToRow
-                                                  InputAdapter
-                                                    Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
-                                                      SubqueryBroadcast [d_date_sk] #5
-                                                        BroadcastExchange #20
-                                                          WholeStageCodegen (1)
-                                                            Project [d_date_sk]
-                                                              Filter [d_week_seq,d_date_sk]
-                                                                Subquery #6
-                                                                  WholeStageCodegen (1)
-                                                                    Project [d_week_seq]
-                                                                      Filter [d_year,d_moy,d_dom]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
-                                                                ColumnarToRow
-                                                                  InputAdapter
-                                                                    Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+                                BroadcastHashJoin [ss_item_sk,ss_item_sk]
+                                  Filter [ss_item_sk]
+                                    ColumnarToRow
+                                      InputAdapter
+                                        Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+                                          SubqueryBroadcast [d_date_sk] #5
+                                            BroadcastExchange #17
+                                              WholeStageCodegen (1)
+                                                Project [d_date_sk]
+                                                  Filter [d_week_seq,d_date_sk]
+                                                    Subquery #6
+                                                      WholeStageCodegen (1)
+                                                        Project [d_week_seq]
+                                                          Filter [d_year,d_moy,d_dom]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
+                                                    ColumnarToRow
+                                                      InputAdapter
+                                                        Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
                                   InputAdapter
-                                    WholeStageCodegen (63)
-                                      Sort [ss_item_sk]
-                                        InputAdapter
-                                          ReusedExchange [ss_item_sk] #4
+                                    ReusedExchange [ss_item_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #20
+                                  ReusedExchange [d_date_sk] #17
                             InputAdapter
-                              ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+                              ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
index c87645f3ad8..d7f8bfd6a12 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
@@ -1,103 +1,94 @@
 == Physical Plan ==
-TakeOrderedAndProject (99)
-+- * BroadcastHashJoin Inner BuildRight (98)
-   :- * Filter (78)
-   :  +- * HashAggregate (77)
-   :     +- Exchange (76)
-   :        +- * HashAggregate (75)
-   :           +- * Project (74)
-   :              +- * BroadcastHashJoin Inner BuildRight (73)
-   :                 :- * Project (63)
-   :                 :  +- * BroadcastHashJoin Inner BuildRight (62)
-   :                 :     :- * SortMergeJoin LeftSemi (60)
-   :                 :     :  :- * Sort (5)
-   :                 :     :  :  +- Exchange (4)
-   :                 :     :  :     +- * Filter (3)
-   :                 :     :  :        +- * ColumnarToRow (2)
-   :                 :     :  :           +- Scan parquet spark_catalog.default.store_sales (1)
-   :                 :     :  +- * Sort (59)
-   :                 :     :     +- Exchange (58)
-   :                 :     :        +- * Project (57)
-   :                 :     :           +- * BroadcastHashJoin Inner BuildRight (56)
-   :                 :     :              :- * Filter (8)
-   :                 :     :              :  +- * ColumnarToRow (7)
-   :                 :     :              :     +- Scan parquet spark_catalog.default.item (6)
-   :                 :     :              +- BroadcastExchange (55)
-   :                 :     :                 +- * SortMergeJoin LeftSemi (54)
-   :                 :     :                    :- * Sort (42)
-   :                 :     :                    :  +- Exchange (41)
-   :                 :     :                    :     +- * HashAggregate (40)
-   :                 :     :                    :        +- Exchange (39)
-   :                 :     :                    :           +- * HashAggregate (38)
-   :                 :     :                    :              +- * Project (37)
-   :                 :     :                    :                 +- * BroadcastHashJoin Inner BuildRight (36)
-   :                 :     :                    :                    :- * Project (14)
-   :                 :     :                    :                    :  +- * BroadcastHashJoin Inner BuildRight (13)
-   :                 :     :                    :                    :     :- * Filter (11)
-   :                 :     :                    :                    :     :  +- * ColumnarToRow (10)
-   :                 :     :                    :                    :     :     +- Scan parquet spark_catalog.default.store_sales (9)
-   :                 :     :                    :                    :     +- ReusedExchange (12)
-   :                 :     :                    :                    +- BroadcastExchange (35)
-   :                 :     :                    :                       +- * SortMergeJoin LeftSemi (34)
-   :                 :     :                    :                          :- * Sort (19)
-   :                 :     :                    :                          :  +- Exchange (18)
-   :                 :     :                    :                          :     +- * Filter (17)
-   :                 :     :                    :                          :        +- * ColumnarToRow (16)
-   :                 :     :                    :                          :           +- Scan parquet spark_catalog.default.item (15)
-   :                 :     :                    :                          +- * Sort (33)
-   :                 :     :                    :                             +- Exchange (32)
-   :                 :     :                    :                                +- * Project (31)
-   :                 :     :                    :                                   +- * BroadcastHashJoin Inner BuildRight (30)
-   :                 :     :                    :                                      :- * Project (25)
-   :                 :     :                    :                                      :  +- * BroadcastHashJoin Inner BuildRight (24)
-   :                 :     :                    :                                      :     :- * Filter (22)
-   :                 :     :                    :                                      :     :  +- * ColumnarToRow (21)
-   :                 :     :                    :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (20)
-   :                 :     :                    :                                      :     +- ReusedExchange (23)
-   :                 :     :                    :                                      +- BroadcastExchange (29)
-   :                 :     :                    :                                         +- * Filter (28)
-   :                 :     :                    :                                            +- * ColumnarToRow (27)
-   :                 :     :                    :                                               +- Scan parquet spark_catalog.default.item (26)
-   :                 :     :                    +- * Sort (53)
-   :                 :     :                       +- Exchange (52)
-   :                 :     :                          +- * Project (51)
-   :                 :     :                             +- * BroadcastHashJoin Inner BuildRight (50)
-   :                 :     :                                :- * Project (48)
-   :                 :     :                                :  +- * BroadcastHashJoin Inner BuildRight (47)
-   :                 :     :                                :     :- * Filter (45)
-   :                 :     :                                :     :  +- * ColumnarToRow (44)
-   :                 :     :                                :     :     +- Scan parquet spark_catalog.default.web_sales (43)
-   :                 :     :                                :     +- ReusedExchange (46)
-   :                 :     :                                +- ReusedExchange (49)
-   :                 :     +- ReusedExchange (61)
-   :                 +- BroadcastExchange (72)
-   :                    +- * SortMergeJoin LeftSemi (71)
-   :                       :- * Sort (68)
-   :                       :  +- Exchange (67)
-   :                       :     +- * Filter (66)
-   :                       :        +- * ColumnarToRow (65)
-   :                       :           +- Scan parquet spark_catalog.default.item (64)
-   :                       +- * Sort (70)
-   :                          +- ReusedExchange (69)
-   +- BroadcastExchange (97)
-      +- * Filter (96)
-         +- * HashAggregate (95)
-            +- Exchange (94)
-               +- * HashAggregate (93)
-                  +- * Project (92)
-                     +- * BroadcastHashJoin Inner BuildRight (91)
-                        :- * Project (89)
-                        :  +- * BroadcastHashJoin Inner BuildRight (88)
-                        :     :- * SortMergeJoin LeftSemi (86)
-                        :     :  :- * Sort (83)
-                        :     :  :  +- Exchange (82)
-                        :     :  :     +- * Filter (81)
-                        :     :  :        +- * ColumnarToRow (80)
-                        :     :  :           +- Scan parquet spark_catalog.default.store_sales (79)
-                        :     :  +- * Sort (85)
-                        :     :     +- ReusedExchange (84)
-                        :     +- ReusedExchange (87)
-                        +- ReusedExchange (90)
+TakeOrderedAndProject (90)
++- * BroadcastHashJoin Inner BuildRight (89)
+   :- * Filter (72)
+   :  +- * HashAggregate (71)
+   :     +- Exchange (70)
+   :        +- * HashAggregate (69)
+   :           +- * Project (68)
+   :              +- * BroadcastHashJoin Inner BuildRight (67)
+   :                 :- * Project (60)
+   :                 :  +- * BroadcastHashJoin Inner BuildRight (59)
+   :                 :     :- * BroadcastHashJoin LeftSemi BuildRight (57)
+   :                 :     :  :- * Filter (3)
+   :                 :     :  :  +- * ColumnarToRow (2)
+   :                 :     :  :     +- Scan parquet spark_catalog.default.store_sales (1)
+   :                 :     :  +- BroadcastExchange (56)
+   :                 :     :     +- * Project (55)
+   :                 :     :        +- * BroadcastHashJoin Inner BuildRight (54)
+   :                 :     :           :- * Filter (6)
+   :                 :     :           :  +- * ColumnarToRow (5)
+   :                 :     :           :     +- Scan parquet spark_catalog.default.item (4)
+   :                 :     :           +- BroadcastExchange (53)
+   :                 :     :              +- * SortMergeJoin LeftSemi (52)
+   :                 :     :                 :- * Sort (40)
+   :                 :     :                 :  +- Exchange (39)
+   :                 :     :                 :     +- * HashAggregate (38)
+   :                 :     :                 :        +- Exchange (37)
+   :                 :     :                 :           +- * HashAggregate (36)
+   :                 :     :                 :              +- * Project (35)
+   :                 :     :                 :                 +- * BroadcastHashJoin Inner BuildRight (34)
+   :                 :     :                 :                    :- * Project (12)
+   :                 :     :                 :                    :  +- * BroadcastHashJoin Inner BuildRight (11)
+   :                 :     :                 :                    :     :- * Filter (9)
+   :                 :     :                 :                    :     :  +- * ColumnarToRow (8)
+   :                 :     :                 :                    :     :     +- Scan parquet spark_catalog.default.store_sales (7)
+   :                 :     :                 :                    :     +- ReusedExchange (10)
+   :                 :     :                 :                    +- BroadcastExchange (33)
+   :                 :     :                 :                       +- * SortMergeJoin LeftSemi (32)
+   :                 :     :                 :                          :- * Sort (17)
+   :                 :     :                 :                          :  +- Exchange (16)
+   :                 :     :                 :                          :     +- * Filter (15)
+   :                 :     :                 :                          :        +- * ColumnarToRow (14)
+   :                 :     :                 :                          :           +- Scan parquet spark_catalog.default.item (13)
+   :                 :     :                 :                          +- * Sort (31)
+   :                 :     :                 :                             +- Exchange (30)
+   :                 :     :                 :                                +- * Project (29)
+   :                 :     :                 :                                   +- * BroadcastHashJoin Inner BuildRight (28)
+   :                 :     :                 :                                      :- * Project (23)
+   :                 :     :                 :                                      :  +- * BroadcastHashJoin Inner BuildRight (22)
+   :                 :     :                 :                                      :     :- * Filter (20)
+   :                 :     :                 :                                      :     :  +- * ColumnarToRow (19)
+   :                 :     :                 :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (18)
+   :                 :     :                 :                                      :     +- ReusedExchange (21)
+   :                 :     :                 :                                      +- BroadcastExchange (27)
+   :                 :     :                 :                                         +- * Filter (26)
+   :                 :     :                 :                                            +- * ColumnarToRow (25)
+   :                 :     :                 :                                               +- Scan parquet spark_catalog.default.item (24)
+   :                 :     :                 +- * Sort (51)
+   :                 :     :                    +- Exchange (50)
+   :                 :     :                       +- * Project (49)
+   :                 :     :                          +- * BroadcastHashJoin Inner BuildRight (48)
+   :                 :     :                             :- * Project (46)
+   :                 :     :                             :  +- * BroadcastHashJoin Inner BuildRight (45)
+   :                 :     :                             :     :- * Filter (43)
+   :                 :     :                             :     :  +- * ColumnarToRow (42)
+   :                 :     :                             :     :     +- Scan parquet spark_catalog.default.web_sales (41)
+   :                 :     :                             :     +- ReusedExchange (44)
+   :                 :     :                             +- ReusedExchange (47)
+   :                 :     +- ReusedExchange (58)
+   :                 +- BroadcastExchange (66)
+   :                    +- * BroadcastHashJoin LeftSemi BuildRight (65)
+   :                       :- * Filter (63)
+   :                       :  +- * ColumnarToRow (62)
+   :                       :     +- Scan parquet spark_catalog.default.item (61)
+   :                       +- ReusedExchange (64)
+   +- BroadcastExchange (88)
+      +- * Filter (87)
+         +- * HashAggregate (86)
+            +- Exchange (85)
+               +- * HashAggregate (84)
+                  +- * Project (83)
+                     +- * BroadcastHashJoin Inner BuildRight (82)
+                        :- * Project (80)
+                        :  +- * BroadcastHashJoin Inner BuildRight (79)
+                        :     :- * BroadcastHashJoin LeftSemi BuildRight (77)
+                        :     :  :- * Filter (75)
+                        :     :  :  +- * ColumnarToRow (74)
+                        :     :  :     +- Scan parquet spark_catalog.default.store_sales (73)
+                        :     :  +- ReusedExchange (76)
+                        :     +- ReusedExchange (78)
+                        +- ReusedExchange (81)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -108,36 +99,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 Condition : isnotnull(ss_item_sk#1)
 
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
 
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
 Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Batched: true
 Location: InMemoryFileIndex []
@@ -145,49 +128,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int>
 
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Condition : isnotnull(ss_item_sk#10)
 
-(12) ReusedExchange [Reuses operator id: 132]
+(10) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#13]
 
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_sold_date_sk#11]
 Right keys [1]: [d_date_sk#13]
 Join type: Inner
 Join condition: None
 
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
 Output [1]: [ss_item_sk#10]
 Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
 
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
 
-(18) Exchange
+(16) Exchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
 
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
 
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
 Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Batched: true
 Location: InMemoryFileIndex []
@@ -195,109 +178,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int>
 
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Condition : isnotnull(cs_item_sk#18)
 
-(23) ReusedExchange [Reuses operator id: 132]
+(21) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#20]
 
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_sold_date_sk#19]
 Right keys [1]: [d_date_sk#20]
 Join type: Inner
 Join condition: None
 
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
 Output [1]: [cs_item_sk#18]
 Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
 
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Condition : isnotnull(i_item_sk#21)
 
-(29) BroadcastExchange
+(27) BroadcastExchange
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
 
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_item_sk#18]
 Right keys [1]: [i_item_sk#21]
 Join type: Inner
 Join condition: None
 
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
 Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(32) Exchange
+(30) Exchange
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
 
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
 
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
 Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
 Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
 Join type: LeftSemi
 Join condition: None
 
-(35) BroadcastExchange
+(33) BroadcastExchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_item_sk#10]
 Right keys [1]: [i_item_sk#14]
 Join type: Inner
 Join condition: None
 
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
 Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
 Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(39) Exchange
+(37) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(41) Exchange
+(39) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
 
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
 Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Batched: true
 Location: InMemoryFileIndex []
@@ -305,166 +288,150 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int>
 
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Condition : isnotnull(ws_item_sk#28)
 
-(46) ReusedExchange [Reuses operator id: 132]
+(44) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#30]
 
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_sold_date_sk#29]
 Right keys [1]: [d_date_sk#30]
 Join type: Inner
 Join condition: None
 
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
 Output [1]: [ws_item_sk#28]
 Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
 
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
 Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_item_sk#28]
 Right keys [1]: [i_item_sk#31]
 Join type: Inner
 Join condition: None
 
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
 Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(52) Exchange
+(50) Exchange
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
 
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
 
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
 Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
 Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
 Join type: LeftSemi
 Join condition: None
 
-(55) BroadcastExchange
+(53) BroadcastExchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
 
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
 Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
 Right keys [3]: [brand_id#25, class_id#26, category_id#27]
 Join type: Inner
 Join condition: None
 
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
 Output [1]: [i_item_sk#6 AS ss_item_sk#35]
 Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
 
-(58) Exchange
+(56) BroadcastExchange
 Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
 
-(59) Sort [codegen id : 20]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(61) ReusedExchange [Reuses operator id: 123]
+(58) ReusedExchange [Reuses operator id: 114]
 Output [1]: [d_date_sk#36]
 
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_sold_date_sk#4]
 Right keys [1]: [d_date_sk#36]
 Join type: Inner
 Join condition: None
 
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
 Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
 Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
 
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Condition : (((isnotnull(i_item_sk#37) AND isnotnull(i_brand_id#38)) AND isnotnull(i_class_id#39)) AND isnotnull(i_category_id#40))
 
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
 Left keys [1]: [i_item_sk#37]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(72) BroadcastExchange
+(66) BroadcastExchange
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
 
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [i_item_sk#37]
 Join type: Inner
 Join condition: None
 
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
 Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
-(76) Exchange
+(70) Exchange
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
-(77) HashAggregate [codegen id : 88]
+(71) HashAggregate [codegen id : 76]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
 Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
-(78) Filter [codegen id : 88]
+(72) Filter [codegen id : 76]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
 Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(79) Scan parquet spark_catalog.default.store_sales
+(73) Scan parquet spark_catalog.default.store_sales
 Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
 Batched: true
 Location: InMemoryFileIndex []
@@ -472,358 +439,346 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(80) ColumnarToRow [codegen id : 44]
+(74) ColumnarToRow [codegen id : 74]
 Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
 
-(81) Filter [codegen id : 44]
+(75) Filter [codegen id : 74]
 Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
 Condition : isnotnull(ss_item_sk#54)
 
-(82) Exchange
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: hashpartitioning(ss_item_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(83) Sort [codegen id : 45]
-Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57]
-Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0
-
-(84) ReusedExchange [Reuses operator id: 58]
+(76) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(85) Sort [codegen id : 63]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(86) SortMergeJoin [codegen id : 86]
+(77) BroadcastHashJoin [codegen id : 74]
 Left keys [1]: [ss_item_sk#54]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(87) ReusedExchange [Reuses operator id: 137]
+(78) ReusedExchange [Reuses operator id: 128]
 Output [1]: [d_date_sk#59]
 
-(88) BroadcastHashJoin [codegen id : 86]
+(79) BroadcastHashJoin [codegen id : 74]
 Left keys [1]: [ss_sold_date_sk#57]
 Right keys [1]: [d_date_sk#59]
 Join type: Inner
 Join condition: None
 
-(89) Project [codegen id : 86]
+(80) Project [codegen id : 74]
 Output [3]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56]
 Input [5]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, d_date_sk#59]
 
-(90) ReusedExchange [Reuses operator id: 72]
+(81) ReusedExchange [Reuses operator id: 66]
 Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
 
-(91) BroadcastHashJoin [codegen id : 86]
+(82) BroadcastHashJoin [codegen id : 74]
 Left keys [1]: [ss_item_sk#54]
 Right keys [1]: [i_item_sk#60]
 Join type: Inner
 Join condition: None
 
-(92) Project [codegen id : 86]
+(83) Project [codegen id : 74]
 Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
 Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63]
 
-(93) HashAggregate [codegen id : 86]
+(84) HashAggregate [codegen id : 74]
 Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
 Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
 Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 
-(94) Exchange
+(85) Exchange
 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
-Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
-(95) HashAggregate [codegen id : 87]
+(86) HashAggregate [codegen id : 75]
 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
 Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
 Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
 
-(96) Filter [codegen id : 87]
+(87) Filter [codegen id : 75]
 Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
 Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(97) BroadcastExchange
+(88) BroadcastExchange
 Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
-Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=16]
+Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=13]
 
-(98) BroadcastHashJoin [codegen id : 88]
+(89) BroadcastHashJoin [codegen id : 76]
 Left keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
 Join type: Inner
 Join condition: None
 
-(99) TakeOrderedAndProject
+(90) TakeOrderedAndProject
 Input [12]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
 Arguments: 100, [i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51, channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
 
 ===== Subqueries =====
 
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
-* HashAggregate (118)
-+- Exchange (117)
-   +- * HashAggregate (116)
-      +- Union (115)
-         :- * Project (104)
-         :  +- * BroadcastHashJoin Inner BuildRight (103)
-         :     :- * ColumnarToRow (101)
-         :     :  +- Scan parquet spark_catalog.default.store_sales (100)
-         :     +- ReusedExchange (102)
-         :- * Project (109)
-         :  +- * BroadcastHashJoin Inner BuildRight (108)
-         :     :- * ColumnarToRow (106)
-         :     :  +- Scan parquet spark_catalog.default.catalog_sales (105)
-         :     +- ReusedExchange (107)
-         +- * Project (114)
-            +- * BroadcastHashJoin Inner BuildRight (113)
-               :- * ColumnarToRow (111)
-               :  +- Scan parquet spark_catalog.default.web_sales (110)
-               +- ReusedExchange (112)
-
-
-(100) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
+* HashAggregate (109)
++- Exchange (108)
+   +- * HashAggregate (107)
+      +- Union (106)
+         :- * Project (95)
+         :  +- * BroadcastHashJoin Inner BuildRight (94)
+         :     :- * ColumnarToRow (92)
+         :     :  +- Scan parquet spark_catalog.default.store_sales (91)
+         :     +- ReusedExchange (93)
+         :- * Project (100)
+         :  +- * BroadcastHashJoin Inner BuildRight (99)
+         :     :- * ColumnarToRow (97)
+         :     :  +- Scan parquet spark_catalog.default.catalog_sales (96)
+         :     +- ReusedExchange (98)
+         +- * Project (105)
+            +- * BroadcastHashJoin Inner BuildRight (104)
+               :- * ColumnarToRow (102)
+               :  +- Scan parquet spark_catalog.default.web_sales (101)
+               +- ReusedExchange (103)
+
+
+(91) Scan parquet spark_catalog.default.store_sales
 Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)]
 ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(101) ColumnarToRow [codegen id : 2]
+(92) ColumnarToRow [codegen id : 2]
 Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77]
 
-(102) ReusedExchange [Reuses operator id: 132]
+(93) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#78]
 
-(103) BroadcastHashJoin [codegen id : 2]
+(94) BroadcastHashJoin [codegen id : 2]
 Left keys [1]: [ss_sold_date_sk#77]
 Right keys [1]: [d_date_sk#78]
 Join type: Inner
 Join condition: None
 
-(104) Project [codegen id : 2]
+(95) Project [codegen id : 2]
 Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80]
 Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78]
 
-(105) Scan parquet spark_catalog.default.catalog_sales
+(96) Scan parquet spark_catalog.default.catalog_sales
 Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)]
 ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
 
-(106) ColumnarToRow [codegen id : 4]
+(97) ColumnarToRow [codegen id : 4]
 Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83]
 
-(107) ReusedExchange [Reuses operator id: 132]
+(98) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#84]
 
-(108) BroadcastHashJoin [codegen id : 4]
+(99) BroadcastHashJoin [codegen id : 4]
 Left keys [1]: [cs_sold_date_sk#83]
 Right keys [1]: [d_date_sk#84]
 Join type: Inner
 Join condition: None
 
-(109) Project [codegen id : 4]
+(100) Project [codegen id : 4]
 Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86]
 Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84]
 
-(110) Scan parquet spark_catalog.default.web_sales
+(101) Scan parquet spark_catalog.default.web_sales
 Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)]
 ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
 
-(111) ColumnarToRow [codegen id : 6]
+(102) ColumnarToRow [codegen id : 6]
 Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89]
 
-(112) ReusedExchange [Reuses operator id: 132]
+(103) ReusedExchange [Reuses operator id: 123]
 Output [1]: [d_date_sk#90]
 
-(113) BroadcastHashJoin [codegen id : 6]
+(104) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [ws_sold_date_sk#89]
 Right keys [1]: [d_date_sk#90]
 Join type: Inner
 Join condition: None
 
-(114) Project [codegen id : 6]
+(105) Project [codegen id : 6]
 Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92]
 Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
 
-(115) Union
+(106) Union
 
-(116) HashAggregate [codegen id : 7]
+(107) HashAggregate [codegen id : 7]
 Input [2]: [quantity#79, list_price#80]
 Keys: []
 Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [2]: [sum#93, count#94]
 Results [2]: [sum#95, count#96]
 
-(117) Exchange
+(108) Exchange
 Input [2]: [sum#95, count#96]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14]
 
-(118) HashAggregate [codegen id : 8]
+(109) HashAggregate [codegen id : 8]
 Input [2]: [sum#95, count#96]
 Keys: []
 Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
 Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
 
-Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
 
-Subquery:3 Hosting operator id = 105 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
+Subquery:3 Hosting operator id = 96 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12
 
-Subquery:4 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
+Subquery:4 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12
 
 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (123)
-+- * Project (122)
-   +- * Filter (121)
-      +- * ColumnarToRow (120)
-         +- Scan parquet spark_catalog.default.date_dim (119)
+BroadcastExchange (114)
++- * Project (113)
+   +- * Filter (112)
+      +- * ColumnarToRow (111)
+         +- Scan parquet spark_catalog.default.date_dim (110)
 
 
-(119) Scan parquet spark_catalog.default.date_dim
+(110) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#36, d_week_seq#99]
 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>
 
-(120) ColumnarToRow [codegen id : 1]
+(111) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#36, d_week_seq#99]
 
-(121) Filter [codegen id : 1]
+(112) Filter [codegen id : 1]
 Input [2]: [d_date_sk#36, d_week_seq#99]
 Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#36))
 
-(122) Project [codegen id : 1]
+(113) Project [codegen id : 1]
 Output [1]: [d_date_sk#36]
 Input [2]: [d_date_sk#36, d_week_seq#99]
 
-(123) BroadcastExchange
+(114) BroadcastExchange
 Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15]
 
-Subquery:6 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
-* Project (127)
-+- * Filter (126)
-   +- * ColumnarToRow (125)
-      +- Scan parquet spark_catalog.default.date_dim (124)
+Subquery:6 Hosting operator id = 112 Hosting Expression = Subquery scalar-subquery#100, [id=#101]
+* Project (118)
++- * Filter (117)
+   +- * ColumnarToRow (116)
+      +- Scan parquet spark_catalog.default.date_dim (115)
 
 
-(124) Scan parquet spark_catalog.default.date_dim
+(115) Scan parquet spark_catalog.default.date_dim
 Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)]
 ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
 
-(125) ColumnarToRow [codegen id : 1]
+(116) ColumnarToRow [codegen id : 1]
 Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 
-(126) Filter [codegen id : 1]
+(117) Filter [codegen id : 1]
 Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16))
 
-(127) Project [codegen id : 1]
+(118) Project [codegen id : 1]
 Output [1]: [d_week_seq#102]
 Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105]
 
-Subquery:7 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (132)
-+- * Project (131)
-   +- * Filter (130)
-      +- * ColumnarToRow (129)
-         +- Scan parquet spark_catalog.default.date_dim (128)
+Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (123)
++- * Project (122)
+   +- * Filter (121)
+      +- * ColumnarToRow (120)
+         +- Scan parquet spark_catalog.default.date_dim (119)
 
 
-(128) Scan parquet spark_catalog.default.date_dim
+(119) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#13, d_year#106]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(129) ColumnarToRow [codegen id : 1]
+(120) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#106]
 
-(130) Filter [codegen id : 1]
+(121) Filter [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#106]
 Condition : (((isnotnull(d_year#106) AND (d_year#106 >= 1998)) AND (d_year#106 <= 2000)) AND isnotnull(d_date_sk#13))
 
-(131) Project [codegen id : 1]
+(122) Project [codegen id : 1]
 Output [1]: [d_date_sk#13]
 Input [2]: [d_date_sk#13, d_year#106]
 
-(132) BroadcastExchange
+(123) BroadcastExchange
 Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
 
-Subquery:8 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
 
-Subquery:9 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:9 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
 
-Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:10 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
 
-Subquery:11 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
-BroadcastExchange (137)
-+- * Project (136)
-   +- * Filter (135)
-      +- * ColumnarToRow (134)
-         +- Scan parquet spark_catalog.default.date_dim (133)
+Subquery:11 Hosting operator id = 73 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58
+BroadcastExchange (128)
++- * Project (127)
+   +- * Filter (126)
+      +- * ColumnarToRow (125)
+         +- Scan parquet spark_catalog.default.date_dim (124)
 
 
-(133) Scan parquet spark_catalog.default.date_dim
+(124) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#59, d_week_seq#107]
 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>
 
-(134) ColumnarToRow [codegen id : 1]
+(125) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#59, d_week_seq#107]
 
-(135) Filter [codegen id : 1]
+(126) Filter [codegen id : 1]
 Input [2]: [d_date_sk#59, d_week_seq#107]
 Condition : ((isnotnull(d_week_seq#107) AND (d_week_seq#107 = Subquery scalar-subquery#108, [id=#109])) AND isnotnull(d_date_sk#59))
 
-(136) Project [codegen id : 1]
+(127) Project [codegen id : 1]
 Output [1]: [d_date_sk#59]
 Input [2]: [d_date_sk#59, d_week_seq#107]
 
-(137) BroadcastExchange
+(128) BroadcastExchange
 Input [1]: [d_date_sk#59]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17]
 
-Subquery:12 Hosting operator id = 135 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
-* Project (141)
-+- * Filter (140)
-   +- * ColumnarToRow (139)
-      +- Scan parquet spark_catalog.default.date_dim (138)
+Subquery:12 Hosting operator id = 126 Hosting Expression = Subquery scalar-subquery#108, [id=#109]
+* Project (132)
++- * Filter (131)
+   +- * ColumnarToRow (130)
+      +- Scan parquet spark_catalog.default.date_dim (129)
 
 
-(138) Scan parquet spark_catalog.default.date_dim
+(129) Scan parquet spark_catalog.default.date_dim
 Output [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)]
 ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int>
 
-(139) ColumnarToRow [codegen id : 1]
+(130) ColumnarToRow [codegen id : 1]
 Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 
-(140) Filter [codegen id : 1]
+(131) Filter [codegen id : 1]
 Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 Condition : (((((isnotnull(d_year#111) AND isnotnull(d_moy#112)) AND isnotnull(d_dom#113)) AND (d_year#111 = 1998)) AND (d_moy#112 = 12)) AND (d_dom#113 = 16))
 
-(141) Project [codegen id : 1]
+(132) Project [codegen id : 1]
 Output [1]: [d_week_seq#110]
 Input [4]: [d_week_seq#110, d_year#111, d_moy#112, d_dom#113]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
index 8c67fff5b14..edd34864986 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
@@ -1,12 +1,12 @@
 TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales]
-  WholeStageCodegen (88)
+  WholeStageCodegen (76)
     BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
       Filter [sales]
         Subquery #4
           WholeStageCodegen (8)
             HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
               InputAdapter
-                Exchange #16
+                Exchange #14
                   WholeStageCodegen (7)
                     HashAggregate [quantity,list_price] [sum,count,sum,count]
                       InputAdapter
@@ -19,7 +19,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                     Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #8
+                                  ReusedExchange [d_date_sk] #7
                           WholeStageCodegen (4)
                             Project [cs_quantity,cs_list_price]
                               BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -28,7 +28,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                     Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #8
+                                  ReusedExchange [d_date_sk] #7
                           WholeStageCodegen (6)
                             Project [ws_quantity,ws_list_price]
                               BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -37,211 +37,184 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                     Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #8
+                                  ReusedExchange [d_date_sk] #7
         HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
           InputAdapter
             Exchange [i_brand_id,i_class_id,i_category_id] #1
-              WholeStageCodegen (43)
+              WholeStageCodegen (37)
                 HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                   Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
                     BroadcastHashJoin [ss_item_sk,i_item_sk]
                       Project [ss_item_sk,ss_quantity,ss_list_price]
                         BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                          SortMergeJoin [ss_item_sk,ss_item_sk]
-                            InputAdapter
-                              WholeStageCodegen (2)
-                                Sort [ss_item_sk]
-                                  InputAdapter
-                                    Exchange [ss_item_sk] #2
-                                      WholeStageCodegen (1)
-                                        Filter [ss_item_sk]
-                                          ColumnarToRow
-                                            InputAdapter
-                                              Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
-                                                SubqueryBroadcast [d_date_sk] #1
-                                                  BroadcastExchange #3
-                                                    WholeStageCodegen (1)
-                                                      Project [d_date_sk]
-                                                        Filter [d_week_seq,d_date_sk]
-                                                          Subquery #2
-                                                            WholeStageCodegen (1)
-                                                              Project [d_week_seq]
-                                                                Filter [d_year,d_moy,d_dom]
-                                                                  ColumnarToRow
-                                                                    InputAdapter
-                                                                      Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
-                                                          ColumnarToRow
-                                                            InputAdapter
-                                                              Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
-                            InputAdapter
-                              WholeStageCodegen (20)
-                                Sort [ss_item_sk]
-                                  InputAdapter
-                                    Exchange [ss_item_sk] #4
-                                      WholeStageCodegen (19)
-                                        Project [i_item_sk]
-                                          BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
-                                            Filter [i_brand_id,i_class_id,i_category_id]
+                          BroadcastHashJoin [ss_item_sk,ss_item_sk]
+                            Filter [ss_item_sk]
+                              ColumnarToRow
+                                InputAdapter
+                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+                                    SubqueryBroadcast [d_date_sk] #1
+                                      BroadcastExchange #2
+                                        WholeStageCodegen (1)
+                                          Project [d_date_sk]
+                                            Filter [d_week_seq,d_date_sk]
+                                              Subquery #2
+                                                WholeStageCodegen (1)
+                                                  Project [d_week_seq]
+                                                    Filter [d_year,d_moy,d_dom]
+                                                      ColumnarToRow
+                                                        InputAdapter
+                                                          Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
                                               ColumnarToRow
                                                 InputAdapter
-                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                            InputAdapter
-                                              BroadcastExchange #5
-                                                WholeStageCodegen (18)
-                                                  SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                                  Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+                            InputAdapter
+                              BroadcastExchange #3
+                                WholeStageCodegen (17)
+                                  Project [i_item_sk]
+                                    BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+                                      Filter [i_brand_id,i_class_id,i_category_id]
+                                        ColumnarToRow
+                                          InputAdapter
+                                            Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                      InputAdapter
+                                        BroadcastExchange #4
+                                          WholeStageCodegen (16)
+                                            SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                              InputAdapter
+                                                WholeStageCodegen (11)
+                                                  Sort [brand_id,class_id,category_id]
                                                     InputAdapter
-                                                      WholeStageCodegen (13)
-                                                        Sort [brand_id,class_id,category_id]
-                                                          InputAdapter
-                                                            Exchange [brand_id,class_id,category_id] #6
-                                                              WholeStageCodegen (12)
-                                                                HashAggregate [brand_id,class_id,category_id]
-                                                                  InputAdapter
-                                                                    Exchange [brand_id,class_id,category_id] #7
-                                                                      WholeStageCodegen (11)
-                                                                        HashAggregate [brand_id,class_id,category_id]
-                                                                          Project [i_brand_id,i_class_id,i_category_id]
-                                                                            BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                                                              Project [ss_item_sk]
-                                                                                BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                  Filter [ss_item_sk]
-                                                                                    ColumnarToRow
-                                                                                      InputAdapter
-                                                                                        Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
-                                                                                          SubqueryBroadcast [d_date_sk] #3
-                                                                                            BroadcastExchange #8
-                                                                                              WholeStageCodegen (1)
-                                                                                                Project [d_date_sk]
-                                                                                                  Filter [d_year,d_date_sk]
-                                                                                                    ColumnarToRow
-                                                                                                      InputAdapter
-                                                                                                        Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
-                                                                                  InputAdapter
-                                                                                    ReusedExchange [d_date_sk] #8
-                                                                              InputAdapter
-                                                                                BroadcastExchange #9
-                                                                                  WholeStageCodegen (10)
-                                                                                    SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                      Exchange [brand_id,class_id,category_id] #5
+                                                        WholeStageCodegen (10)
+                                                          HashAggregate [brand_id,class_id,category_id]
+                                                            InputAdapter
+                                                              Exchange [brand_id,class_id,category_id] #6
+                                                                WholeStageCodegen (9)
+                                                                  HashAggregate [brand_id,class_id,category_id]
+                                                                    Project [i_brand_id,i_class_id,i_category_id]
+                                                                      BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                                                        Project [ss_item_sk]
+                                                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                            Filter [ss_item_sk]
+                                                                              ColumnarToRow
+                                                                                InputAdapter
+                                                                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+                                                                                    SubqueryBroadcast [d_date_sk] #3
+                                                                                      BroadcastExchange #7
+                                                                                        WholeStageCodegen (1)
+                                                                                          Project [d_date_sk]
+                                                                                            Filter [d_year,d_date_sk]
+                                                                                              ColumnarToRow
+                                                                                                InputAdapter
+                                                                                                  Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+                                                                            InputAdapter
+                                                                              ReusedExchange [d_date_sk] #7
+                                                                        InputAdapter
+                                                                          BroadcastExchange #8
+                                                                            WholeStageCodegen (8)
+                                                                              SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                                                InputAdapter
+                                                                                  WholeStageCodegen (3)
+                                                                                    Sort [i_brand_id,i_class_id,i_category_id]
                                                                                       InputAdapter
-                                                                                        WholeStageCodegen (5)
-                                                                                          Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                            InputAdapter
-                                                                                              Exchange [i_brand_id,i_class_id,i_category_id] #10
-                                                                                                WholeStageCodegen (4)
-                                                                                                  Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                                                                                    ColumnarToRow
-                                                                                                      InputAdapter
-                                                                                                        Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                        Exchange [i_brand_id,i_class_id,i_category_id] #9
+                                                                                          WholeStageCodegen (2)
+                                                                                            Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                              ColumnarToRow
+                                                                                                InputAdapter
+                                                                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                InputAdapter
+                                                                                  WholeStageCodegen (7)
+                                                                                    Sort [i_brand_id,i_class_id,i_category_id]
                                                                                       InputAdapter
-                                                                                        WholeStageCodegen (9)
-                                                                                          Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                            InputAdapter
-                                                                                              Exchange [i_brand_id,i_class_id,i_category_id] #11
-                                                                                                WholeStageCodegen (8)
-                                                                                                  Project [i_brand_id,i_class_id,i_category_id]
-                                                                                                    BroadcastHashJoin [cs_item_sk,i_item_sk]
-                                                                                                      Project [cs_item_sk]
-                                                                                                        BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                                                                          Filter [cs_item_sk]
-                                                                                                            ColumnarToRow
-                                                                                                              InputAdapter
-                                                                                                                Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
-                                                                                                                  ReusedSubquery [d_date_sk] #3
+                                                                                        Exchange [i_brand_id,i_class_id,i_category_id] #10
+                                                                                          WholeStageCodegen (6)
+                                                                                            Project [i_brand_id,i_class_id,i_category_id]
+                                                                                              BroadcastHashJoin [cs_item_sk,i_item_sk]
+                                                                                                Project [cs_item_sk]
+                                                                                                  BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+                                                                                                    Filter [cs_item_sk]
+                                                                                                      ColumnarToRow
+                                                                                                        InputAdapter
+                                                                                                          Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+                                                                                                            ReusedSubquery [d_date_sk] #3
+                                                                                                    InputAdapter
+                                                                                                      ReusedExchange [d_date_sk] #7
+                                                                                                InputAdapter
+                                                                                                  BroadcastExchange #11
+                                                                                                    WholeStageCodegen (5)
+                                                                                                      Filter [i_item_sk]
+                                                                                                        ColumnarToRow
                                                                                                           InputAdapter
-                                                                                                            ReusedExchange [d_date_sk] #8
-                                                                                                      InputAdapter
-                                                                                                        BroadcastExchange #12
-                                                                                                          WholeStageCodegen (7)
-                                                                                                            Filter [i_item_sk]
-                                                                                                              ColumnarToRow
-                                                                                                                InputAdapter
-                                                                                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                            Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                              InputAdapter
+                                                WholeStageCodegen (15)
+                                                  Sort [i_brand_id,i_class_id,i_category_id]
                                                     InputAdapter
-                                                      WholeStageCodegen (17)
-                                                        Sort [i_brand_id,i_class_id,i_category_id]
-                                                          InputAdapter
-                                                            Exchange [i_brand_id,i_class_id,i_category_id] #13
-                                                              WholeStageCodegen (16)
-                                                                Project [i_brand_id,i_class_id,i_category_id]
-                                                                  BroadcastHashJoin [ws_item_sk,i_item_sk]
-                                                                    Project [ws_item_sk]
-                                                                      BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                                                        Filter [ws_item_sk]
-                                                                          ColumnarToRow
-                                                                            InputAdapter
-                                                                              Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
-                                                                                ReusedSubquery [d_date_sk] #3
-                                                                        InputAdapter
-                                                                          ReusedExchange [d_date_sk] #8
-                                                                    InputAdapter
-                                                                      ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12
+                                                      Exchange [i_brand_id,i_class_id,i_category_id] #12
+                                                        WholeStageCodegen (14)
+                                                          Project [i_brand_id,i_class_id,i_category_id]
+                                                            BroadcastHashJoin [ws_item_sk,i_item_sk]
+                                                              Project [ws_item_sk]
+                                                                BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+                                                                  Filter [ws_item_sk]
+                                                                    ColumnarToRow
+                                                                      InputAdapter
+                                                                        Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+                                                                          ReusedSubquery [d_date_sk] #3
+                                                                  InputAdapter
+                                                                    ReusedExchange [d_date_sk] #7
+                                                              InputAdapter
+                                                                ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
                           InputAdapter
-                            ReusedExchange [d_date_sk] #3
+                            ReusedExchange [d_date_sk] #2
                       InputAdapter
-                        BroadcastExchange #14
-                          WholeStageCodegen (42)
-                            SortMergeJoin [i_item_sk,ss_item_sk]
-                              InputAdapter
-                                WholeStageCodegen (23)
-                                  Sort [i_item_sk]
-                                    InputAdapter
-                                      Exchange [i_item_sk] #15
-                                        WholeStageCodegen (22)
-                                          Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                            ColumnarToRow
-                                              InputAdapter
-                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                        BroadcastExchange #13
+                          WholeStageCodegen (36)
+                            BroadcastHashJoin [i_item_sk,ss_item_sk]
+                              Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                ColumnarToRow
+                                  InputAdapter
+                                    Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
                               InputAdapter
-                                WholeStageCodegen (41)
-                                  Sort [ss_item_sk]
-                                    InputAdapter
-                                      ReusedExchange [ss_item_sk] #4
+                                ReusedExchange [ss_item_sk] #3
       InputAdapter
-        BroadcastExchange #17
-          WholeStageCodegen (87)
+        BroadcastExchange #15
+          WholeStageCodegen (75)
             Filter [sales]
               ReusedSubquery [average_sales] #4
               HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                 InputAdapter
-                  Exchange [i_brand_id,i_class_id,i_category_id] #18
-                    WholeStageCodegen (86)
+                  Exchange [i_brand_id,i_class_id,i_category_id] #16
+                    WholeStageCodegen (74)
                       HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                         Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
                           BroadcastHashJoin [ss_item_sk,i_item_sk]
                             Project [ss_item_sk,ss_quantity,ss_list_price]
                               BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                SortMergeJoin [ss_item_sk,ss_item_sk]
-                                  InputAdapter
-                                    WholeStageCodegen (45)
-                                      Sort [ss_item_sk]
-                                        InputAdapter
-                                          Exchange [ss_item_sk] #19
-                                            WholeStageCodegen (44)
-                                              Filter [ss_item_sk]
-                                                ColumnarToRow
-                                                  InputAdapter
-                                                    Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
-                                                      SubqueryBroadcast [d_date_sk] #5
-                                                        BroadcastExchange #20
-                                                          WholeStageCodegen (1)
-                                                            Project [d_date_sk]
-                                                              Filter [d_week_seq,d_date_sk]
-                                                                Subquery #6
-                                                                  WholeStageCodegen (1)
-                                                                    Project [d_week_seq]
-                                                                      Filter [d_year,d_moy,d_dom]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
-                                                                ColumnarToRow
-                                                                  InputAdapter
-                                                                    Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+                                BroadcastHashJoin [ss_item_sk,ss_item_sk]
+                                  Filter [ss_item_sk]
+                                    ColumnarToRow
+                                      InputAdapter
+                                        Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+                                          SubqueryBroadcast [d_date_sk] #5
+                                            BroadcastExchange #17
+                                              WholeStageCodegen (1)
+                                                Project [d_date_sk]
+                                                  Filter [d_week_seq,d_date_sk]
+                                                    Subquery #6
+                                                      WholeStageCodegen (1)
+                                                        Project [d_week_seq]
+                                                          Filter [d_year,d_moy,d_dom]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
+                                                    ColumnarToRow
+                                                      InputAdapter
+                                                        Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
                                   InputAdapter
-                                    WholeStageCodegen (63)
-                                      Sort [ss_item_sk]
-                                        InputAdapter
-                                          ReusedExchange [ss_item_sk] #4
+                                    ReusedExchange [ss_item_sk] #3
                                 InputAdapter
-                                  ReusedExchange [d_date_sk] #20
+                                  ReusedExchange [d_date_sk] #17
                             InputAdapter
-                              ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+                              ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
index f3db50d8f15..9527bf6a061 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
@@ -1,147 +1,135 @@
 == Physical Plan ==
-TakeOrderedAndProject (143)
-+- * HashAggregate (142)
-   +- Exchange (141)
-      +- * HashAggregate (140)
-         +- Union (139)
-            :- * HashAggregate (118)
-            :  +- Exchange (117)
-            :     +- * HashAggregate (116)
-            :        +- Union (115)
-            :           :- * Filter (78)
-            :           :  +- * HashAggregate (77)
-            :           :     +- Exchange (76)
-            :           :        +- * HashAggregate (75)
-            :           :           +- * Project (74)
-            :           :              +- * BroadcastHashJoin Inner BuildRight (73)
-            :           :                 :- * Project (63)
-            :           :                 :  +- * BroadcastHashJoin Inner BuildRight (62)
-            :           :                 :     :- * SortMergeJoin LeftSemi (60)
-            :           :                 :     :  :- * Sort (5)
-            :           :                 :     :  :  +- Exchange (4)
-            :           :                 :     :  :     +- * Filter (3)
-            :           :                 :     :  :        +- * ColumnarToRow (2)
-            :           :                 :     :  :           +- Scan parquet spark_catalog.default.store_sales (1)
-            :           :                 :     :  +- * Sort (59)
-            :           :                 :     :     +- Exchange (58)
-            :           :                 :     :        +- * Project (57)
-            :           :                 :     :           +- * BroadcastHashJoin Inner BuildRight (56)
-            :           :                 :     :              :- * Filter (8)
-            :           :                 :     :              :  +- * ColumnarToRow (7)
-            :           :                 :     :              :     +- Scan parquet spark_catalog.default.item (6)
-            :           :                 :     :              +- BroadcastExchange (55)
-            :           :                 :     :                 +- * SortMergeJoin LeftSemi (54)
-            :           :                 :     :                    :- * Sort (42)
-            :           :                 :     :                    :  +- Exchange (41)
-            :           :                 :     :                    :     +- * HashAggregate (40)
-            :           :                 :     :                    :        +- Exchange (39)
-            :           :                 :     :                    :           +- * HashAggregate (38)
-            :           :                 :     :                    :              +- * Project (37)
-            :           :                 :     :                    :                 +- * BroadcastHashJoin Inner BuildRight (36)
-            :           :                 :     :                    :                    :- * Project (14)
-            :           :                 :     :                    :                    :  +- * BroadcastHashJoin Inner BuildRight (13)
-            :           :                 :     :                    :                    :     :- * Filter (11)
-            :           :                 :     :                    :                    :     :  +- * ColumnarToRow (10)
-            :           :                 :     :                    :                    :     :     +- Scan parquet spark_catalog.default.store_sales (9)
-            :           :                 :     :                    :                    :     +- ReusedExchange (12)
-            :           :                 :     :                    :                    +- BroadcastExchange (35)
-            :           :                 :     :                    :                       +- * SortMergeJoin LeftSemi (34)
-            :           :                 :     :                    :                          :- * Sort (19)
-            :           :                 :     :                    :                          :  +- Exchange (18)
-            :           :                 :     :                    :                          :     +- * Filter (17)
-            :           :                 :     :                    :                          :        +- * ColumnarToRow (16)
-            :           :                 :     :                    :                          :           +- Scan parquet spark_catalog.default.item (15)
-            :           :                 :     :                    :                          +- * Sort (33)
-            :           :                 :     :                    :                             +- Exchange (32)
-            :           :                 :     :                    :                                +- * Project (31)
-            :           :                 :     :                    :                                   +- * BroadcastHashJoin Inner BuildRight (30)
-            :           :                 :     :                    :                                      :- * Project (25)
-            :           :                 :     :                    :                                      :  +- * BroadcastHashJoin Inner BuildRight (24)
-            :           :                 :     :                    :                                      :     :- * Filter (22)
-            :           :                 :     :                    :                                      :     :  +- * ColumnarToRow (21)
-            :           :                 :     :                    :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (20)
-            :           :                 :     :                    :                                      :     +- ReusedExchange (23)
-            :           :                 :     :                    :                                      +- BroadcastExchange (29)
-            :           :                 :     :                    :                                         +- * Filter (28)
-            :           :                 :     :                    :                                            +- * ColumnarToRow (27)
-            :           :                 :     :                    :                                               +- Scan parquet spark_catalog.default.item (26)
-            :           :                 :     :                    +- * Sort (53)
-            :           :                 :     :                       +- Exchange (52)
-            :           :                 :     :                          +- * Project (51)
-            :           :                 :     :                             +- * BroadcastHashJoin Inner BuildRight (50)
-            :           :                 :     :                                :- * Project (48)
-            :           :                 :     :                                :  +- * BroadcastHashJoin Inner BuildRight (47)
-            :           :                 :     :                                :     :- * Filter (45)
-            :           :                 :     :                                :     :  +- * ColumnarToRow (44)
-            :           :                 :     :                                :     :     +- Scan parquet spark_catalog.default.web_sales (43)
-            :           :                 :     :                                :     +- ReusedExchange (46)
-            :           :                 :     :                                +- ReusedExchange (49)
-            :           :                 :     +- ReusedExchange (61)
-            :           :                 +- BroadcastExchange (72)
-            :           :                    +- * SortMergeJoin LeftSemi (71)
-            :           :                       :- * Sort (68)
-            :           :                       :  +- Exchange (67)
-            :           :                       :     +- * Filter (66)
-            :           :                       :        +- * ColumnarToRow (65)
-            :           :                       :           +- Scan parquet spark_catalog.default.item (64)
-            :           :                       +- * Sort (70)
-            :           :                          +- ReusedExchange (69)
-            :           :- * Filter (96)
-            :           :  +- * HashAggregate (95)
-            :           :     +- Exchange (94)
-            :           :        +- * HashAggregate (93)
-            :           :           +- * Project (92)
-            :           :              +- * BroadcastHashJoin Inner BuildRight (91)
-            :           :                 :- * Project (89)
-            :           :                 :  +- * BroadcastHashJoin Inner BuildRight (88)
-            :           :                 :     :- * SortMergeJoin LeftSemi (86)
-            :           :                 :     :  :- * Sort (83)
-            :           :                 :     :  :  +- Exchange (82)
-            :           :                 :     :  :     +- * Filter (81)
-            :           :                 :     :  :        +- * ColumnarToRow (80)
-            :           :                 :     :  :           +- Scan parquet spark_catalog.default.catalog_sales (79)
-            :           :                 :     :  +- * Sort (85)
-            :           :                 :     :     +- ReusedExchange (84)
-            :           :                 :     +- ReusedExchange (87)
-            :           :                 +- ReusedExchange (90)
-            :           +- * Filter (114)
-            :              +- * HashAggregate (113)
-            :                 +- Exchange (112)
-            :                    +- * HashAggregate (111)
-            :                       +- * Project (110)
-            :                          +- * BroadcastHashJoin Inner BuildRight (109)
-            :                             :- * Project (107)
-            :                             :  +- * BroadcastHashJoin Inner BuildRight (106)
-            :                             :     :- * SortMergeJoin LeftSemi (104)
-            :                             :     :  :- * Sort (101)
-            :                             :     :  :  +- Exchange (100)
-            :                             :     :  :     +- * Filter (99)
-            :                             :     :  :        +- * ColumnarToRow (98)
-            :                             :     :  :           +- Scan parquet spark_catalog.default.web_sales (97)
-            :                             :     :  +- * Sort (103)
-            :                             :     :     +- ReusedExchange (102)
-            :                             :     +- ReusedExchange (105)
-            :                             +- ReusedExchange (108)
-            :- * HashAggregate (123)
-            :  +- Exchange (122)
-            :     +- * HashAggregate (121)
-            :        +- * HashAggregate (120)
-            :           +- ReusedExchange (119)
-            :- * HashAggregate (128)
-            :  +- Exchange (127)
-            :     +- * HashAggregate (126)
-            :        +- * HashAggregate (125)
-            :           +- ReusedExchange (124)
-            :- * HashAggregate (133)
-            :  +- Exchange (132)
-            :     +- * HashAggregate (131)
-            :        +- * HashAggregate (130)
-            :           +- ReusedExchange (129)
-            +- * HashAggregate (138)
-               +- Exchange (137)
-                  +- * HashAggregate (136)
-                     +- * HashAggregate (135)
-                        +- ReusedExchange (134)
+TakeOrderedAndProject (131)
++- * HashAggregate (130)
+   +- Exchange (129)
+      +- * HashAggregate (128)
+         +- Union (127)
+            :- * HashAggregate (106)
+            :  +- Exchange (105)
+            :     +- * HashAggregate (104)
+            :        +- Union (103)
+            :           :- * Filter (72)
+            :           :  +- * HashAggregate (71)
+            :           :     +- Exchange (70)
+            :           :        +- * HashAggregate (69)
+            :           :           +- * Project (68)
+            :           :              +- * BroadcastHashJoin Inner BuildRight (67)
+            :           :                 :- * Project (60)
+            :           :                 :  +- * BroadcastHashJoin Inner BuildRight (59)
+            :           :                 :     :- * BroadcastHashJoin LeftSemi BuildRight (57)
+            :           :                 :     :  :- * Filter (3)
+            :           :                 :     :  :  +- * ColumnarToRow (2)
+            :           :                 :     :  :     +- Scan parquet spark_catalog.default.store_sales (1)
+            :           :                 :     :  +- BroadcastExchange (56)
+            :           :                 :     :     +- * Project (55)
+            :           :                 :     :        +- * BroadcastHashJoin Inner BuildRight (54)
+            :           :                 :     :           :- * Filter (6)
+            :           :                 :     :           :  +- * ColumnarToRow (5)
+            :           :                 :     :           :     +- Scan parquet spark_catalog.default.item (4)
+            :           :                 :     :           +- BroadcastExchange (53)
+            :           :                 :     :              +- * SortMergeJoin LeftSemi (52)
+            :           :                 :     :                 :- * Sort (40)
+            :           :                 :     :                 :  +- Exchange (39)
+            :           :                 :     :                 :     +- * HashAggregate (38)
+            :           :                 :     :                 :        +- Exchange (37)
+            :           :                 :     :                 :           +- * HashAggregate (36)
+            :           :                 :     :                 :              +- * Project (35)
+            :           :                 :     :                 :                 +- * BroadcastHashJoin Inner BuildRight (34)
+            :           :                 :     :                 :                    :- * Project (12)
+            :           :                 :     :                 :                    :  +- * BroadcastHashJoin Inner BuildRight (11)
+            :           :                 :     :                 :                    :     :- * Filter (9)
+            :           :                 :     :                 :                    :     :  +- * ColumnarToRow (8)
+            :           :                 :     :                 :                    :     :     +- Scan parquet spark_catalog.default.store_sales (7)
+            :           :                 :     :                 :                    :     +- ReusedExchange (10)
+            :           :                 :     :                 :                    +- BroadcastExchange (33)
+            :           :                 :     :                 :                       +- * SortMergeJoin LeftSemi (32)
+            :           :                 :     :                 :                          :- * Sort (17)
+            :           :                 :     :                 :                          :  +- Exchange (16)
+            :           :                 :     :                 :                          :     +- * Filter (15)
+            :           :                 :     :                 :                          :        +- * ColumnarToRow (14)
+            :           :                 :     :                 :                          :           +- Scan parquet spark_catalog.default.item (13)
+            :           :                 :     :                 :                          +- * Sort (31)
+            :           :                 :     :                 :                             +- Exchange (30)
+            :           :                 :     :                 :                                +- * Project (29)
+            :           :                 :     :                 :                                   +- * BroadcastHashJoin Inner BuildRight (28)
+            :           :                 :     :                 :                                      :- * Project (23)
+            :           :                 :     :                 :                                      :  +- * BroadcastHashJoin Inner BuildRight (22)
+            :           :                 :     :                 :                                      :     :- * Filter (20)
+            :           :                 :     :                 :                                      :     :  +- * ColumnarToRow (19)
+            :           :                 :     :                 :                                      :     :     +- Scan parquet spark_catalog.default.catalog_sales (18)
+            :           :                 :     :                 :                                      :     +- ReusedExchange (21)
+            :           :                 :     :                 :                                      +- BroadcastExchange (27)
+            :           :                 :     :                 :                                         +- * Filter (26)
+            :           :                 :     :                 :                                            +- * ColumnarToRow (25)
+            :           :                 :     :                 :                                               +- Scan parquet spark_catalog.default.item (24)
+            :           :                 :     :                 +- * Sort (51)
+            :           :                 :     :                    +- Exchange (50)
+            :           :                 :     :                       +- * Project (49)
+            :           :                 :     :                          +- * BroadcastHashJoin Inner BuildRight (48)
+            :           :                 :     :                             :- * Project (46)
+            :           :                 :     :                             :  +- * BroadcastHashJoin Inner BuildRight (45)
+            :           :                 :     :                             :     :- * Filter (43)
+            :           :                 :     :                             :     :  +- * ColumnarToRow (42)
+            :           :                 :     :                             :     :     +- Scan parquet spark_catalog.default.web_sales (41)
+            :           :                 :     :                             :     +- ReusedExchange (44)
+            :           :                 :     :                             +- ReusedExchange (47)
+            :           :                 :     +- ReusedExchange (58)
+            :           :                 +- BroadcastExchange (66)
+            :           :                    +- * BroadcastHashJoin LeftSemi BuildRight (65)
+            :           :                       :- * Filter (63)
+            :           :                       :  +- * ColumnarToRow (62)
+            :           :                       :     +- Scan parquet spark_catalog.default.item (61)
+            :           :                       +- ReusedExchange (64)
+            :           :- * Filter (87)
+            :           :  +- * HashAggregate (86)
+            :           :     +- Exchange (85)
+            :           :        +- * HashAggregate (84)
+            :           :           +- * Project (83)
+            :           :              +- * BroadcastHashJoin Inner BuildRight (82)
+            :           :                 :- * Project (80)
+            :           :                 :  +- * BroadcastHashJoin Inner BuildRight (79)
+            :           :                 :     :- * BroadcastHashJoin LeftSemi BuildRight (77)
+            :           :                 :     :  :- * Filter (75)
+            :           :                 :     :  :  +- * ColumnarToRow (74)
+            :           :                 :     :  :     +- Scan parquet spark_catalog.default.catalog_sales (73)
+            :           :                 :     :  +- ReusedExchange (76)
+            :           :                 :     +- ReusedExchange (78)
+            :           :                 +- ReusedExchange (81)
+            :           +- * Filter (102)
+            :              +- * HashAggregate (101)
+            :                 +- Exchange (100)
+            :                    +- * HashAggregate (99)
+            :                       +- * Project (98)
+            :                          +- * BroadcastHashJoin Inner BuildRight (97)
+            :                             :- * Project (95)
+            :                             :  +- * BroadcastHashJoin Inner BuildRight (94)
+            :                             :     :- * BroadcastHashJoin LeftSemi BuildRight (92)
+            :                             :     :  :- * Filter (90)
+            :                             :     :  :  +- * ColumnarToRow (89)
+            :                             :     :  :     +- Scan parquet spark_catalog.default.web_sales (88)
+            :                             :     :  +- ReusedExchange (91)
+            :                             :     +- ReusedExchange (93)
+            :                             +- ReusedExchange (96)
+            :- * HashAggregate (111)
+            :  +- Exchange (110)
+            :     +- * HashAggregate (109)
+            :        +- * HashAggregate (108)
+            :           +- ReusedExchange (107)
+            :- * HashAggregate (116)
+            :  +- Exchange (115)
+            :     +- * HashAggregate (114)
+            :        +- * HashAggregate (113)
+            :           +- ReusedExchange (112)
+            :- * HashAggregate (121)
+            :  +- Exchange (120)
+            :     +- * HashAggregate (119)
+            :        +- * HashAggregate (118)
+            :           +- ReusedExchange (117)
+            +- * HashAggregate (126)
+               +- Exchange (125)
+                  +- * HashAggregate (124)
+                     +- * HashAggregate (123)
+                        +- ReusedExchange (122)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -152,36 +140,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(2) ColumnarToRow [codegen id : 1]
+(2) ColumnarToRow [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 
-(3) Filter [codegen id : 1]
+(3) Filter [codegen id : 37]
 Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
 Condition : isnotnull(ss_item_sk#1)
 
-(4) Exchange
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(5) Sort [codegen id : 2]
-Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
-Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
-
-(6) Scan parquet spark_catalog.default.item
+(4) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(7) ColumnarToRow [codegen id : 19]
+(5) ColumnarToRow [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 
-(8) Filter [codegen id : 19]
+(6) Filter [codegen id : 17]
 Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
 Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9))
 
-(9) Scan parquet spark_catalog.default.store_sales
+(7) Scan parquet spark_catalog.default.store_sales
 Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Batched: true
 Location: InMemoryFileIndex []
@@ -189,49 +169,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int>
 
-(10) ColumnarToRow [codegen id : 11]
+(8) ColumnarToRow [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 
-(11) Filter [codegen id : 11]
+(9) Filter [codegen id : 9]
 Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
 Condition : isnotnull(ss_item_sk#10)
 
-(12) ReusedExchange [Reuses operator id: 177]
+(10) ReusedExchange [Reuses operator id: 165]
 Output [1]: [d_date_sk#13]
 
-(13) BroadcastHashJoin [codegen id : 11]
+(11) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_sold_date_sk#11]
 Right keys [1]: [d_date_sk#13]
 Join type: Inner
 Join condition: None
 
-(14) Project [codegen id : 11]
+(12) Project [codegen id : 9]
 Output [1]: [ss_item_sk#10]
 Input [3]: [ss_item_sk#10, ss_sold_date_sk#11, d_date_sk#13]
 
-(15) Scan parquet spark_catalog.default.item
+(13) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(16) ColumnarToRow [codegen id : 4]
+(14) ColumnarToRow [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(17) Filter [codegen id : 4]
+(15) Filter [codegen id : 2]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Condition : (((isnotnull(i_item_sk#14) AND isnotnull(i_brand_id#15)) AND isnotnull(i_class_id#16)) AND isnotnull(i_category_id#17))
 
-(18) Exchange
+(16) Exchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=2]
+Arguments: hashpartitioning(coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17), 5), ENSURE_REQUIREMENTS, [plan_id=1]
 
-(19) Sort [codegen id : 5]
+(17) Sort [codegen id : 3]
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 Arguments: [coalesce(i_brand_id#15, 0) ASC NULLS FIRST, isnull(i_brand_id#15) ASC NULLS FIRST, coalesce(i_class_id#16, 0) ASC NULLS FIRST, isnull(i_class_id#16) ASC NULLS FIRST, coalesce(i_category_id#17, 0) ASC NULLS FIRST, isnull(i_category_id#17) ASC NULLS FIRST], false, 0
 
-(20) Scan parquet spark_catalog.default.catalog_sales
+(18) Scan parquet spark_catalog.default.catalog_sales
 Output [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Batched: true
 Location: InMemoryFileIndex []
@@ -239,109 +219,109 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int>
 
-(21) ColumnarToRow [codegen id : 8]
+(19) ColumnarToRow [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 
-(22) Filter [codegen id : 8]
+(20) Filter [codegen id : 6]
 Input [2]: [cs_item_sk#18, cs_sold_date_sk#19]
 Condition : isnotnull(cs_item_sk#18)
 
-(23) ReusedExchange [Reuses operator id: 177]
+(21) ReusedExchange [Reuses operator id: 165]
 Output [1]: [d_date_sk#20]
 
-(24) BroadcastHashJoin [codegen id : 8]
+(22) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_sold_date_sk#19]
 Right keys [1]: [d_date_sk#20]
 Join type: Inner
 Join condition: None
 
-(25) Project [codegen id : 8]
+(23) Project [codegen id : 6]
 Output [1]: [cs_item_sk#18]
 Input [3]: [cs_item_sk#18, cs_sold_date_sk#19, d_date_sk#20]
 
-(26) Scan parquet spark_catalog.default.item
+(24) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(27) ColumnarToRow [codegen id : 7]
+(25) ColumnarToRow [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(28) Filter [codegen id : 7]
+(26) Filter [codegen id : 5]
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 Condition : isnotnull(i_item_sk#21)
 
-(29) BroadcastExchange
+(27) BroadcastExchange
 Input [4]: [i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
 
-(30) BroadcastHashJoin [codegen id : 8]
+(28) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [cs_item_sk#18]
 Right keys [1]: [i_item_sk#21]
 Join type: Inner
 Join condition: None
 
-(31) Project [codegen id : 8]
+(29) Project [codegen id : 6]
 Output [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Input [5]: [cs_item_sk#18, i_item_sk#21, i_brand_id#22, i_class_id#23, i_category_id#24]
 
-(32) Exchange
+(30) Exchange
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
-Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Arguments: hashpartitioning(coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24), 5), ENSURE_REQUIREMENTS, [plan_id=3]
 
-(33) Sort [codegen id : 9]
+(31) Sort [codegen id : 7]
 Input [3]: [i_brand_id#22, i_class_id#23, i_category_id#24]
 Arguments: [coalesce(i_brand_id#22, 0) ASC NULLS FIRST, isnull(i_brand_id#22) ASC NULLS FIRST, coalesce(i_class_id#23, 0) ASC NULLS FIRST, isnull(i_class_id#23) ASC NULLS FIRST, coalesce(i_category_id#24, 0) ASC NULLS FIRST, isnull(i_category_id#24) ASC NULLS FIRST], false, 0
 
-(34) SortMergeJoin [codegen id : 10]
+(32) SortMergeJoin [codegen id : 8]
 Left keys [6]: [coalesce(i_brand_id#15, 0), isnull(i_brand_id#15), coalesce(i_class_id#16, 0), isnull(i_class_id#16), coalesce(i_category_id#17, 0), isnull(i_category_id#17)]
 Right keys [6]: [coalesce(i_brand_id#22, 0), isnull(i_brand_id#22), coalesce(i_class_id#23, 0), isnull(i_class_id#23), coalesce(i_category_id#24, 0), isnull(i_category_id#24)]
 Join type: LeftSemi
 Join condition: None
 
-(35) BroadcastExchange
+(33) BroadcastExchange
 Input [4]: [i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(36) BroadcastHashJoin [codegen id : 11]
+(34) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_item_sk#10]
 Right keys [1]: [i_item_sk#14]
 Join type: Inner
 Join condition: None
 
-(37) Project [codegen id : 11]
+(35) Project [codegen id : 9]
 Output [3]: [i_brand_id#15 AS brand_id#25, i_class_id#16 AS class_id#26, i_category_id#17 AS category_id#27]
 Input [5]: [ss_item_sk#10, i_item_sk#14, i_brand_id#15, i_class_id#16, i_category_id#17]
 
-(38) HashAggregate [codegen id : 11]
+(36) HashAggregate [codegen id : 9]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(39) Exchange
+(37) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(40) HashAggregate [codegen id : 12]
+(38) HashAggregate [codegen id : 10]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Keys [3]: [brand_id#25, class_id#26, category_id#27]
 Functions: []
 Aggregate Attributes: []
 Results [3]: [brand_id#25, class_id#26, category_id#27]
 
-(41) Exchange
+(39) Exchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Arguments: hashpartitioning(coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27), 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(42) Sort [codegen id : 13]
+(40) Sort [codegen id : 11]
 Input [3]: [brand_id#25, class_id#26, category_id#27]
 Arguments: [coalesce(brand_id#25, 0) ASC NULLS FIRST, isnull(brand_id#25) ASC NULLS FIRST, coalesce(class_id#26, 0) ASC NULLS FIRST, isnull(class_id#26) ASC NULLS FIRST, coalesce(category_id#27, 0) ASC NULLS FIRST, isnull(category_id#27) ASC NULLS FIRST], false, 0
 
-(43) Scan parquet spark_catalog.default.web_sales
+(41) Scan parquet spark_catalog.default.web_sales
 Output [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Batched: true
 Location: InMemoryFileIndex []
@@ -349,166 +329,150 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int>
 
-(44) ColumnarToRow [codegen id : 16]
+(42) ColumnarToRow [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 
-(45) Filter [codegen id : 16]
+(43) Filter [codegen id : 14]
 Input [2]: [ws_item_sk#28, ws_sold_date_sk#29]
 Condition : isnotnull(ws_item_sk#28)
 
-(46) ReusedExchange [Reuses operator id: 177]
+(44) ReusedExchange [Reuses operator id: 165]
 Output [1]: [d_date_sk#30]
 
-(47) BroadcastHashJoin [codegen id : 16]
+(45) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_sold_date_sk#29]
 Right keys [1]: [d_date_sk#30]
 Join type: Inner
 Join condition: None
 
-(48) Project [codegen id : 16]
+(46) Project [codegen id : 14]
 Output [1]: [ws_item_sk#28]
 Input [3]: [ws_item_sk#28, ws_sold_date_sk#29, d_date_sk#30]
 
-(49) ReusedExchange [Reuses operator id: 29]
+(47) ReusedExchange [Reuses operator id: 27]
 Output [4]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(50) BroadcastHashJoin [codegen id : 16]
+(48) BroadcastHashJoin [codegen id : 14]
 Left keys [1]: [ws_item_sk#28]
 Right keys [1]: [i_item_sk#31]
 Join type: Inner
 Join condition: None
 
-(51) Project [codegen id : 16]
+(49) Project [codegen id : 14]
 Output [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Input [5]: [ws_item_sk#28, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34]
 
-(52) Exchange
+(50) Exchange
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
-Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Arguments: hashpartitioning(coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34), 5), ENSURE_REQUIREMENTS, [plan_id=7]
 
-(53) Sort [codegen id : 17]
+(51) Sort [codegen id : 15]
 Input [3]: [i_brand_id#32, i_class_id#33, i_category_id#34]
 Arguments: [coalesce(i_brand_id#32, 0) ASC NULLS FIRST, isnull(i_brand_id#32) ASC NULLS FIRST, coalesce(i_class_id#33, 0) ASC NULLS FIRST, isnull(i_class_id#33) ASC NULLS FIRST, coalesce(i_category_id#34, 0) ASC NULLS FIRST, isnull(i_category_id#34) ASC NULLS FIRST], false, 0
 
-(54) SortMergeJoin [codegen id : 18]
+(52) SortMergeJoin [codegen id : 16]
 Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)]
 Right keys [6]: [coalesce(i_brand_id#32, 0), isnull(i_brand_id#32), coalesce(i_class_id#33, 0), isnull(i_class_id#33), coalesce(i_category_id#34, 0), isnull(i_category_id#34)]
 Join type: LeftSemi
 Join condition: None
 
-(55) BroadcastExchange
+(53) BroadcastExchange
 Input [3]: [brand_id#25, class_id#26, category_id#27]
-Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=9]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=8]
 
-(56) BroadcastHashJoin [codegen id : 19]
+(54) BroadcastHashJoin [codegen id : 17]
 Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
 Right keys [3]: [brand_id#25, class_id#26, category_id#27]
 Join type: Inner
 Join condition: None
 
-(57) Project [codegen id : 19]
+(55) Project [codegen id : 17]
 Output [1]: [i_item_sk#6 AS ss_item_sk#35]
 Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27]
 
-(58) Exchange
-Input [1]: [ss_item_sk#35]
-Arguments: hashpartitioning(ss_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=10]
-
-(59) Sort [codegen id : 20]
+(56) BroadcastExchange
 Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9]
 
-(60) SortMergeJoin [codegen id : 43]
+(57) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(61) ReusedExchange [Reuses operator id: 172]
+(58) ReusedExchange [Reuses operator id: 160]
 Output [1]: [d_date_sk#36]
 
-(62) BroadcastHashJoin [codegen id : 43]
+(59) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_sold_date_sk#4]
 Right keys [1]: [d_date_sk#36]
 Join type: Inner
 Join condition: None
 
-(63) Project [codegen id : 43]
+(60) Project [codegen id : 37]
 Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3]
 Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#36]
 
-(64) Scan parquet spark_catalog.default.item
+(61) Scan parquet spark_catalog.default.item
 Output [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
 
-(65) ColumnarToRow [codegen id : 22]
+(62) ColumnarToRow [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(66) Filter [codegen id : 22]
+(63) Filter [codegen id : 36]
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 Condition : isnotnull(i_item_sk#37)
 
-(67) Exchange
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: hashpartitioning(i_item_sk#37, 5), ENSURE_REQUIREMENTS, [plan_id=11]
-
-(68) Sort [codegen id : 23]
-Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: [i_item_sk#37 ASC NULLS FIRST], false, 0
-
-(69) ReusedExchange [Reuses operator id: 58]
+(64) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(70) Sort [codegen id : 41]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(71) SortMergeJoin [codegen id : 42]
+(65) BroadcastHashJoin [codegen id : 36]
 Left keys [1]: [i_item_sk#37]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(72) BroadcastExchange
+(66) BroadcastExchange
 Input [4]: [i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10]
 
-(73) BroadcastHashJoin [codegen id : 43]
+(67) BroadcastHashJoin [codegen id : 37]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [i_item_sk#37]
 Join type: Inner
 Join condition: None
 
-(74) Project [codegen id : 43]
+(68) Project [codegen id : 37]
 Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_id#38, i_class_id#39, i_category_id#40]
 
-(75) HashAggregate [codegen id : 43]
+(69) HashAggregate [codegen id : 37]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
-(76) Exchange
+(70) Exchange
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
-Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
-(77) HashAggregate [codegen id : 44]
+(71) HashAggregate [codegen id : 38]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
 Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
-(78) Filter [codegen id : 44]
+(72) Filter [codegen id : 38]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
 Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(79) Scan parquet spark_catalog.default.catalog_sales
+(73) Scan parquet spark_catalog.default.catalog_sales
 Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
 Batched: true
 Location: InMemoryFileIndex []
@@ -516,83 +480,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)>
 
-(80) ColumnarToRow [codegen id : 45]
+(74) ColumnarToRow [codegen id : 75]
 Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
 
-(81) Filter [codegen id : 45]
+(75) Filter [codegen id : 75]
 Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
 Condition : isnotnull(cs_item_sk#54)
 
-(82) Exchange
-Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
-Arguments: hashpartitioning(cs_item_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=14]
-
-(83) Sort [codegen id : 46]
-Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57]
-Arguments: [cs_item_sk#54 ASC NULLS FIRST], false, 0
-
-(84) ReusedExchange [Reuses operator id: 58]
+(76) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(85) Sort [codegen id : 64]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(86) SortMergeJoin [codegen id : 87]
+(77) BroadcastHashJoin [codegen id : 75]
 Left keys [1]: [cs_item_sk#54]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(87) ReusedExchange [Reuses operator id: 172]
+(78) ReusedExchange [Reuses operator id: 160]
 Output [1]: [d_date_sk#58]
 
-(88) BroadcastHashJoin [codegen id : 87]
+(79) BroadcastHashJoin [codegen id : 75]
 Left keys [1]: [cs_sold_date_sk#57]
 Right keys [1]: [d_date_sk#58]
 Join type: Inner
 Join condition: None
 
-(89) Project [codegen id : 87]
+(80) Project [codegen id : 75]
 Output [3]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56]
 Input [5]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, d_date_sk#58]
 
-(90) ReusedExchange [Reuses operator id: 72]
+(81) ReusedExchange [Reuses operator id: 66]
 Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62]
 
-(91) BroadcastHashJoin [codegen id : 87]
+(82) BroadcastHashJoin [codegen id : 75]
 Left keys [1]: [cs_item_sk#54]
 Right keys [1]: [i_item_sk#59]
 Join type: Inner
 Join condition: None
 
-(92) Project [codegen id : 87]
+(83) Project [codegen id : 75]
 Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
 Input [7]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62]
 
-(93) HashAggregate [codegen id : 87]
+(84) HashAggregate [codegen id : 75]
 Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
 Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65]
 Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
 
-(94) Exchange
+(85) Exchange
 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
-Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
-(95) HashAggregate [codegen id : 88]
+(86) HashAggregate [codegen id : 76]
 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
 Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
 Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70]
 Results [6]: [catalog AS channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73]
 
-(96) Filter [codegen id : 88]
+(87) Filter [codegen id : 76]
 Input [6]: [channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73]
 Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(97) Scan parquet spark_catalog.default.web_sales
+(88) Scan parquet spark_catalog.default.web_sales
 Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
 Batched: true
 Location: InMemoryFileIndex []
@@ -600,455 +552,443 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_so
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)>
 
-(98) ColumnarToRow [codegen id : 89]
+(89) ColumnarToRow [codegen id : 113]
 Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
 
-(99) Filter [codegen id : 89]
+(90) Filter [codegen id : 113]
 Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
 Condition : isnotnull(ws_item_sk#74)
 
-(100) Exchange
-Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
-Arguments: hashpartitioning(ws_item_sk#74, 5), ENSURE_REQUIREMENTS, [plan_id=16]
-
-(101) Sort [codegen id : 90]
-Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77]
-Arguments: [ws_item_sk#74 ASC NULLS FIRST], false, 0
-
-(102) ReusedExchange [Reuses operator id: 58]
+(91) ReusedExchange [Reuses operator id: 56]
 Output [1]: [ss_item_sk#35]
 
-(103) Sort [codegen id : 108]
-Input [1]: [ss_item_sk#35]
-Arguments: [ss_item_sk#35 ASC NULLS FIRST], false, 0
-
-(104) SortMergeJoin [codegen id : 131]
+(92) BroadcastHashJoin [codegen id : 113]
 Left keys [1]: [ws_item_sk#74]
 Right keys [1]: [ss_item_sk#35]
 Join type: LeftSemi
 Join condition: None
 
-(105) ReusedExchange [Reuses operator id: 172]
+(93) ReusedExchange [Reuses operator id: 160]
 Output [1]: [d_date_sk#78]
 
-(106) BroadcastHashJoin [codegen id : 131]
+(94) BroadcastHashJoin [codegen id : 113]
 Left keys [1]: [ws_sold_date_sk#77]
 Right keys [1]: [d_date_sk#78]
 Join type: Inner
 Join condition: None
 
-(107) Project [codegen id : 131]
+(95) Project [codegen id : 113]
 Output [3]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76]
 Input [5]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, d_date_sk#78]
 
-(108) ReusedExchange [Reuses operator id: 72]
+(96) ReusedExchange [Reuses operator id: 66]
 Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82]
 
-(109) BroadcastHashJoin [codegen id : 131]
+(97) BroadcastHashJoin [codegen id : 113]
 Left keys [1]: [ws_item_sk#74]
 Right keys [1]: [i_item_sk#79]
 Join type: Inner
 Join condition: None
 
-(110) Project [codegen id : 131]
+(98) Project [codegen id : 113]
 Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82]
 Input [7]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82]
 
-(111) HashAggregate [codegen id : 131]
+(99) HashAggregate [codegen id : 113]
 Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82]
 Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
 Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)]
 Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85]
 Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
 
-(112) Exchange
+(100) Exchange
 Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
-Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=17]
+Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [plan_id=13]
 
-(113) HashAggregate [codegen id : 132]
+(101) HashAggregate [codegen id : 114]
 Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
 Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
 Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)]
 Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90]
 Results [6]: [web AS channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93]
 
-(114) Filter [codegen id : 132]
+(102) Filter [codegen id : 114]
 Input [6]: [channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93]
 Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6))))
 
-(115) Union
+(103) Union
 
-(116) HashAggregate [codegen id : 133]
+(104) HashAggregate [codegen id : 115]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
 Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)]
 Aggregate Attributes [3]: [sum#94, isEmpty#95, sum#96]
 Results [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 
-(117) Exchange
+(105) Exchange
 Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=18]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=14]
 
-(118) HashAggregate [codegen id : 134]
+(106) HashAggregate [codegen id : 116]
 Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum(sales#50), sum(number_sales#51)]
 Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101]
 Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
 
-(119) ReusedExchange [Reuses operator id: 117]
+(107) ReusedExchange [Reuses operator id: 105]
 Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 
-(120) HashAggregate [codegen id : 268]
+(108) HashAggregate [codegen id : 232]
 Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum(sales#50), sum(number_sales#51)]
 Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101]
 Results [5]: [channel#49, i_brand_id#38, i_class_id#39, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
 
-(121) HashAggregate [codegen id : 268]
+(109) HashAggregate [codegen id : 232]
 Input [5]: [channel#49, i_brand_id#38, i_class_id#39, sum_sales#102, number_sales#103]
 Keys [3]: [channel#49, i_brand_id#38, i_class_id#39]
 Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
 Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106]
 Results [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109]
 
-(122) Exchange
+(110) Exchange
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=19]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=15]
 
-(123) HashAggregate [codegen id : 269]
+(111) HashAggregate [codegen id : 233]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, sum#107, isEmpty#108, sum#109]
 Keys [3]: [channel#49, i_brand_id#38, i_class_id#39]
 Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
 Aggregate Attributes [2]: [sum(sum_sales#102)#110, sum(number_sales#103)#111]
 Results [6]: [channel#49, i_brand_id#38, i_class_id#39, null AS i_category_id#112, sum(sum_sales#102)#110 AS sum(sum_sales)#113, sum(number_sales#103)#111 AS sum(number_sales)#114]
 
-(124) ReusedExchange [Reuses operator id: 117]
+(112) ReusedExchange [Reuses operator id: 105]
 Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 
-(125) HashAggregate [codegen id : 403]
+(113) HashAggregate [codegen id : 349]
 Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum(sales#50), sum(number_sales#51)]
 Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101]
 Results [4]: [channel#49, i_brand_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
 
-(126) HashAggregate [codegen id : 403]
+(114) HashAggregate [codegen id : 349]
 Input [4]: [channel#49, i_brand_id#38, sum_sales#102, number_sales#103]
 Keys [2]: [channel#49, i_brand_id#38]
 Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
 Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117]
 Results [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120]
 
-(127) Exchange
+(115) Exchange
 Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=20]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=16]
 
-(128) HashAggregate [codegen id : 404]
+(116) HashAggregate [codegen id : 350]
 Input [5]: [channel#49, i_brand_id#38, sum#118, isEmpty#119, sum#120]
 Keys [2]: [channel#49, i_brand_id#38]
 Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
 Aggregate Attributes [2]: [sum(sum_sales#102)#121, sum(number_sales#103)#122]
 Results [6]: [channel#49, i_brand_id#38, null AS i_class_id#123, null AS i_category_id#124, sum(sum_sales#102)#121 AS sum(sum_sales)#125, sum(number_sales#103)#122 AS sum(number_sales)#126]
 
-(129) ReusedExchange [Reuses operator id: 117]
+(117) ReusedExchange [Reuses operator id: 105]
 Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 
-(130) HashAggregate [codegen id : 538]
+(118) HashAggregate [codegen id : 466]
 Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum(sales#50), sum(number_sales#51)]
 Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101]
 Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
 
-(131) HashAggregate [codegen id : 538]
+(119) HashAggregate [codegen id : 466]
 Input [3]: [channel#49, sum_sales#102, number_sales#103]
 Keys [1]: [channel#49]
 Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
 Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129]
 Results [4]: [channel#49, sum#130, isEmpty#131, sum#132]
 
-(132) Exchange
+(120) Exchange
 Input [4]: [channel#49, sum#130, isEmpty#131, sum#132]
-Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=21]
+Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, [plan_id=17]
 
-(133) HashAggregate [codegen id : 539]
+(121) HashAggregate [codegen id : 467]
 Input [4]: [channel#49, sum#130, isEmpty#131, sum#132]
 Keys [1]: [channel#49]
 Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
 Aggregate Attributes [2]: [sum(sum_sales#102)#133, sum(number_sales#103)#134]
 Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null AS i_category_id#137, sum(sum_sales#102)#133 AS sum(sum_sales)#138, sum(number_sales#103)#134 AS sum(number_sales)#139]
 
-(134) ReusedExchange [Reuses operator id: 117]
+(122) ReusedExchange [Reuses operator id: 105]
 Output [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 
-(135) HashAggregate [codegen id : 673]
+(123) HashAggregate [codegen id : 583]
 Input [7]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum#97, isEmpty#98, sum#99]
 Keys [4]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40]
 Functions [2]: [sum(sales#50), sum(number_sales#51)]
 Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101]
 Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103]
 
-(136) HashAggregate [codegen id : 673]
+(124) HashAggregate [codegen id : 583]
 Input [2]: [sum_sales#102, number_sales#103]
 Keys: []
 Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)]
 Aggregate Attributes [3]: [sum#140, isEmpty#141, sum#142]
 Results [3]: [sum#143, isEmpty#144, sum#145]
 
-(137) Exchange
+(125) Exchange
 Input [3]: [sum#143, isEmpty#144, sum#145]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=22]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18]
 
-(138) HashAggregate [codegen id : 674]
+(126) HashAggregate [codegen id : 584]
 Input [3]: [sum#143, isEmpty#144, sum#145]
 Keys: []
 Functions [2]: [sum(sum_sales#102), sum(number_sales#103)]
 Aggregate Attributes [2]: [sum(sum_sales#102)#146, sum(number_sales#103)#147]
 Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#150, null AS i_category_id#151, sum(sum_sales#102)#146 AS sum(sum_sales)#152, sum(number_sales#103)#147 AS sum(number_sales)#153]
 
-(139) Union
+(127) Union
 
-(140) HashAggregate [codegen id : 675]
+(128) HashAggregate [codegen id : 585]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 Functions: []
 Aggregate Attributes: []
 Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 
-(141) Exchange
+(129) Exchange
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
-Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=23]
+Arguments: hashpartitioning(channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, [plan_id=19]
 
-(142) HashAggregate [codegen id : 676]
+(130) HashAggregate [codegen id : 586]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 Keys [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 Functions: []
 Aggregate Attributes: []
 Results [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 
-(143) TakeOrderedAndProject
+(131) TakeOrderedAndProject
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 Arguments: 100, [channel#49 ASC NULLS FIRST, i_brand_id#38 ASC NULLS FIRST, i_class_id#39 ASC NULLS FIRST, i_category_id#40 ASC NULLS FIRST], [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum_sales#102, number_sales#103]
 
 ===== Subqueries =====
 
-Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
-* HashAggregate (162)
-+- Exchange (161)
-   +- * HashAggregate (160)
-      +- Union (159)
-         :- * Project (148)
-         :  +- * BroadcastHashJoin Inner BuildRight (147)
-         :     :- * ColumnarToRow (145)
-         :     :  +- Scan parquet spark_catalog.default.store_sales (144)
-         :     +- ReusedExchange (146)
-         :- * Project (153)
-         :  +- * BroadcastHashJoin Inner BuildRight (152)
-         :     :- * ColumnarToRow (150)
-         :     :  +- Scan parquet spark_catalog.default.catalog_sales (149)
-         :     +- ReusedExchange (151)
-         +- * Project (158)
-            +- * BroadcastHashJoin Inner BuildRight (157)
-               :- * ColumnarToRow (155)
-               :  +- Scan parquet spark_catalog.default.web_sales (154)
-               +- ReusedExchange (156)
-
-
-(144) Scan parquet spark_catalog.default.store_sales
+Subquery:1 Hosting operator id = 72 Hosting Expression = Subquery scalar-subquery#52, [id=#53]
+* HashAggregate (150)
++- Exchange (149)
+   +- * HashAggregate (148)
+      +- Union (147)
+         :- * Project (136)
+         :  +- * BroadcastHashJoin Inner BuildRight (135)
+         :     :- * ColumnarToRow (133)
+         :     :  +- Scan parquet spark_catalog.default.store_sales (132)
+         :     +- ReusedExchange (134)
+         :- * Project (141)
+         :  +- * BroadcastHashJoin Inner BuildRight (140)
+         :     :- * ColumnarToRow (138)
+         :     :  +- Scan parquet spark_catalog.default.catalog_sales (137)
+         :     +- ReusedExchange (139)
+         +- * Project (146)
+            +- * BroadcastHashJoin Inner BuildRight (145)
+               :- * ColumnarToRow (143)
+               :  +- Scan parquet spark_catalog.default.web_sales (142)
+               +- ReusedExchange (144)
+
+
+(132) Scan parquet spark_catalog.default.store_sales
 Output [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)]
 ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
 
-(145) ColumnarToRow [codegen id : 2]
+(133) ColumnarToRow [codegen id : 2]
 Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156]
 
-(146) ReusedExchange [Reuses operator id: 177]
+(134) ReusedExchange [Reuses operator id: 165]
 Output [1]: [d_date_sk#157]
 
-(147) BroadcastHashJoin [codegen id : 2]
+(135) BroadcastHashJoin [codegen id : 2]
 Left keys [1]: [ss_sold_date_sk#156]
 Right keys [1]: [d_date_sk#157]
 Join type: Inner
 Join condition: None
 
-(148) Project [codegen id : 2]
+(136) Project [codegen id : 2]
 Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159]
 Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157]
 
-(149) Scan parquet spark_catalog.default.catalog_sales
+(137) Scan parquet spark_catalog.default.catalog_sales
 Output [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)]
 ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
 
-(150) ColumnarToRow [codegen id : 4]
+(138) ColumnarToRow [codegen id : 4]
 Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162]
 
-(151) ReusedExchange [Reuses operator id: 167]
+(139) ReusedExchange [Reuses operator id: 155]
 Output [1]: [d_date_sk#164]
 
-(152) BroadcastHashJoin [codegen id : 4]
+(140) BroadcastHashJoin [codegen id : 4]
 Left keys [1]: [cs_sold_date_sk#162]
 Right keys [1]: [d_date_sk#164]
 Join type: Inner
 Join condition: None
 
-(153) Project [codegen id : 4]
+(141) Project [codegen id : 4]
 Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166]
 Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164]
 
-(154) Scan parquet spark_catalog.default.web_sales
+(142) Scan parquet spark_catalog.default.web_sales
 Output [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169]
 Batched: true
 Location: InMemoryFileIndex []
 PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)]
 ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
 
-(155) ColumnarToRow [codegen id : 6]
+(143) ColumnarToRow [codegen id : 6]
 Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169]
 
-(156) ReusedExchange [Reuses operator id: 167]
+(144) ReusedExchange [Reuses operator id: 155]
 Output [1]: [d_date_sk#170]
 
-(157) BroadcastHashJoin [codegen id : 6]
+(145) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [ws_sold_date_sk#169]
 Right keys [1]: [d_date_sk#170]
 Join type: Inner
 Join condition: None
 
-(158) Project [codegen id : 6]
+(146) Project [codegen id : 6]
 Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172]
 Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170]
 
-(159) Union
+(147) Union
 
-(160) HashAggregate [codegen id : 7]
+(148) HashAggregate [codegen id : 7]
 Input [2]: [quantity#158, list_price#159]
 Keys: []
 Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
 Aggregate Attributes [2]: [sum#173, count#174]
 Results [2]: [sum#175, count#176]
 
-(161) Exchange
+(149) Exchange
 Input [2]: [sum#175, count#176]
-Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=20]
 
-(162) HashAggregate [codegen id : 8]
+(150) HashAggregate [codegen id : 8]
 Input [2]: [sum#175, count#176]
 Keys: []
 Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
 Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177]
 Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178]
 
-Subquery:2 Hosting operator id = 144 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12
+Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12
 
-Subquery:3 Hosting operator id = 149 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163
-BroadcastExchange (167)
-+- * Project (166)
-   +- * Filter (165)
-      +- * ColumnarToRow (164)
-         +- Scan parquet spark_catalog.default.date_dim (163)
+Subquery:3 Hosting operator id = 137 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163
+BroadcastExchange (155)
++- * Project (154)
+   +- * Filter (153)
+      +- * ColumnarToRow (152)
+         +- Scan parquet spark_catalog.default.date_dim (151)
 
 
-(163) Scan parquet spark_catalog.default.date_dim
+(151) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#164, d_year#179]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(164) ColumnarToRow [codegen id : 1]
+(152) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#164, d_year#179]
 
-(165) Filter [codegen id : 1]
+(153) Filter [codegen id : 1]
 Input [2]: [d_date_sk#164, d_year#179]
 Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#164))
 
-(166) Project [codegen id : 1]
+(154) Project [codegen id : 1]
 Output [1]: [d_date_sk#164]
 Input [2]: [d_date_sk#164, d_year#179]
 
-(167) BroadcastExchange
+(155) BroadcastExchange
 Input [1]: [d_date_sk#164]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21]
 
-Subquery:4 Hosting operator id = 154 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163
+Subquery:4 Hosting operator id = 142 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163
 
 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (172)
-+- * Project (171)
-   +- * Filter (170)
-      +- * ColumnarToRow (169)
-         +- Scan parquet spark_catalog.default.date_dim (168)
+BroadcastExchange (160)
++- * Project (159)
+   +- * Filter (158)
+      +- * ColumnarToRow (157)
+         +- Scan parquet spark_catalog.default.date_dim (156)
 
 
-(168) Scan parquet spark_catalog.default.date_dim
+(156) Scan parquet spark_catalog.default.date_dim
 Output [3]: [d_date_sk#36, d_year#180, d_moy#181]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
 
-(169) ColumnarToRow [codegen id : 1]
+(157) ColumnarToRow [codegen id : 1]
 Input [3]: [d_date_sk#36, d_year#180, d_moy#181]
 
-(170) Filter [codegen id : 1]
+(158) Filter [codegen id : 1]
 Input [3]: [d_date_sk#36, d_year#180, d_moy#181]
 Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 = 2000)) AND (d_moy#181 = 11)) AND isnotnull(d_date_sk#36))
 
-(171) Project [codegen id : 1]
+(159) Project [codegen id : 1]
 Output [1]: [d_date_sk#36]
 Input [3]: [d_date_sk#36, d_year#180, d_moy#181]
 
-(172) BroadcastExchange
+(160) BroadcastExchange
 Input [1]: [d_date_sk#36]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=26]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=22]
 
-Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
-BroadcastExchange (177)
-+- * Project (176)
-   +- * Filter (175)
-      +- * ColumnarToRow (174)
-         +- Scan parquet spark_catalog.default.date_dim (173)
+Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (165)
++- * Project (164)
+   +- * Filter (163)
+      +- * ColumnarToRow (162)
+         +- Scan parquet spark_catalog.default.date_dim (161)
 
 
-(173) Scan parquet spark_catalog.default.date_dim
+(161) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#13, d_year#182]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(174) ColumnarToRow [codegen id : 1]
+(162) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#182]
 
-(175) Filter [codegen id : 1]
+(163) Filter [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#182]
 Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#13))
 
-(176) Project [codegen id : 1]
+(164) Project [codegen id : 1]
 Output [1]: [d_date_sk#13]
 Input [2]: [d_date_sk#13, d_year#182]
 
-(177) BroadcastExchange
+(165) BroadcastExchange
 Input [1]: [d_date_sk#13]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=27]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23]
 
-Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
+Subquery:7 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#12
 
-Subquery:8 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
+Subquery:8 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12
 
-Subquery:9 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:9 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
 
-Subquery:10 Hosting operator id = 79 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5
+Subquery:10 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5
 
-Subquery:11 Hosting operator id = 114 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
+Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53]
 
-Subquery:12 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5
+Subquery:12 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt
index c0788a25e93..84c70b046c7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt
@@ -1,27 +1,27 @@
 TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales]
-  WholeStageCodegen (676)
+  WholeStageCodegen (586)
     HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales]
       InputAdapter
         Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1
-          WholeStageCodegen (675)
+          WholeStageCodegen (585)
             HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales]
               InputAdapter
                 Union
-                  WholeStageCodegen (134)
+                  WholeStageCodegen (116)
                     HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
                       InputAdapter
                         Exchange [channel,i_brand_id,i_class_id,i_category_id] #2
-                          WholeStageCodegen (133)
+                          WholeStageCodegen (115)
                             HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
                               InputAdapter
                                 Union
-                                  WholeStageCodegen (44)
+                                  WholeStageCodegen (38)
                                     Filter [sales]
                                       Subquery #3
                                         WholeStageCodegen (8)
                                           HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
                                             InputAdapter
-                                              Exchange #18
+                                              Exchange #16
                                                 WholeStageCodegen (7)
                                                   HashAggregate [quantity,list_price] [sum,count,sum,count]
                                                     InputAdapter
@@ -34,7 +34,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                                                   Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
                                                                     ReusedSubquery [d_date_sk] #2
                                                               InputAdapter
-                                                                ReusedExchange [d_date_sk] #10
+                                                                ReusedExchange [d_date_sk] #9
                                                         WholeStageCodegen (4)
                                                           Project [cs_quantity,cs_list_price]
                                                             BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
@@ -42,7 +42,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                                                 InputAdapter
                                                                   Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
                                                                     SubqueryBroadcast [d_date_sk] #4
-                                                                      BroadcastExchange #19
+                                                                      BroadcastExchange #17
                                                                         WholeStageCodegen (1)
                                                                           Project [d_date_sk]
                                                                             Filter [d_year,d_date_sk]
@@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                                                                 InputAdapter
                                                                                   Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
                                                               InputAdapter
-                                                                ReusedExchange [d_date_sk] #19
+                                                                ReusedExchange [d_date_sk] #17
                                                         WholeStageCodegen (6)
                                                           Project [ws_quantity,ws_list_price]
                                                             BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
@@ -59,256 +59,220 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                                                   Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
                                                                     ReusedSubquery [d_date_sk] #4
                                                               InputAdapter
-                                                                ReusedExchange [d_date_sk] #19
+                                                                ReusedExchange [d_date_sk] #17
                                       HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #3
-                                            WholeStageCodegen (43)
+                                            WholeStageCodegen (37)
                                               HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                                                 Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id]
                                                   BroadcastHashJoin [ss_item_sk,i_item_sk]
                                                     Project [ss_item_sk,ss_quantity,ss_list_price]
                                                       BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                        SortMergeJoin [ss_item_sk,ss_item_sk]
-                                                          InputAdapter
-                                                            WholeStageCodegen (2)
-                                                              Sort [ss_item_sk]
-                                                                InputAdapter
-                                                                  Exchange [ss_item_sk] #4
-                                                                    WholeStageCodegen (1)
-                                                                      Filter [ss_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
-                                                                              SubqueryBroadcast [d_date_sk] #1
-                                                                                BroadcastExchange #5
-                                                                                  WholeStageCodegen (1)
-                                                                                    Project [d_date_sk]
-                                                                                      Filter [d_year,d_moy,d_date_sk]
-                                                                                        ColumnarToRow
-                                                                                          InputAdapter
-                                                                                            Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
-                                                          InputAdapter
-                                                            WholeStageCodegen (20)
-                                                              Sort [ss_item_sk]
-                                                                InputAdapter
-                                                                  Exchange [ss_item_sk] #6
-                                                                    WholeStageCodegen (19)
-                                                                      Project [i_item_sk]
-                                                                        BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
-                                                                          Filter [i_brand_id,i_class_id,i_category_id]
+                                                        BroadcastHashJoin [ss_item_sk,ss_item_sk]
+                                                          Filter [ss_item_sk]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+                                                                  SubqueryBroadcast [d_date_sk] #1
+                                                                    BroadcastExchange #4
+                                                                      WholeStageCodegen (1)
+                                                                        Project [d_date_sk]
+                                                                          Filter [d_year,d_moy,d_date_sk]
                                                                             ColumnarToRow
                                                                               InputAdapter
-                                                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                                                          InputAdapter
-                                                                            BroadcastExchange #7
-                                                                              WholeStageCodegen (18)
-                                                                                SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                                                                Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+                                                          InputAdapter
+                                                            BroadcastExchange #5
+                                                              WholeStageCodegen (17)
+                                                                Project [i_item_sk]
+                                                                  BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+                                                                    Filter [i_brand_id,i_class_id,i_category_id]
+                                                                      ColumnarToRow
+                                                                        InputAdapter
+                                                                          Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                    InputAdapter
+                                                                      BroadcastExchange #6
+                                                                        WholeStageCodegen (16)
+                                                                          SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+                                                                            InputAdapter
+                                                                              WholeStageCodegen (11)
+                                                                                Sort [brand_id,class_id,category_id]
                                                                                   InputAdapter
-                                                                                    WholeStageCodegen (13)
-                                                                                      Sort [brand_id,class_id,category_id]
-                                                                                        InputAdapter
-                                                                                          Exchange [brand_id,class_id,category_id] #8
-                                                                                            WholeStageCodegen (12)
-                                                                                              HashAggregate [brand_id,class_id,category_id]
-                                                                                                InputAdapter
-                                                                                                  Exchange [brand_id,class_id,category_id] #9
-                                                                                                    WholeStageCodegen (11)
-                                                                                                      HashAggregate [brand_id,class_id,category_id]
-                                                                                                        Project [i_brand_id,i_class_id,i_category_id]
-                                                                                                          BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                                                                                            Project [ss_item_sk]
-                                                                                                              BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                                                Filter [ss_item_sk]
-                                                                                                                  ColumnarToRow
-                                                                                                                    InputAdapter
-                                                                                                                      Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
-                                                                                                                        SubqueryBroadcast [d_date_sk] #2
-                                                                                                                          BroadcastExchange #10
-                                                                                                                            WholeStageCodegen (1)
-                                                                                                                              Project [d_date_sk]
-                                                                                                                                Filter [d_year,d_date_sk]
-                                                                                                                                  ColumnarToRow
-                                                                                                                                    InputAdapter
-                                                                                                                                      Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
-                                                                                                                InputAdapter
-                                                                                                                  ReusedExchange [d_date_sk] #10
-                                                                                                            InputAdapter
-                                                                                                              BroadcastExchange #11
-                                                                                                                WholeStageCodegen (10)
-                                                                                                                  SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                                                    Exchange [brand_id,class_id,category_id] #7
+                                                                                      WholeStageCodegen (10)
+                                                                                        HashAggregate [brand_id,class_id,category_id]
+                                                                                          InputAdapter
+                                                                                            Exchange [brand_id,class_id,category_id] #8
+                                                                                              WholeStageCodegen (9)
+                                                                                                HashAggregate [brand_id,class_id,category_id]
+                                                                                                  Project [i_brand_id,i_class_id,i_category_id]
+                                                                                                    BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                                                                                      Project [ss_item_sk]
+                                                                                                        BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                                                          Filter [ss_item_sk]
+                                                                                                            ColumnarToRow
+                                                                                                              InputAdapter
+                                                                                                                Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+                                                                                                                  SubqueryBroadcast [d_date_sk] #2
+                                                                                                                    BroadcastExchange #9
+                                                                                                                      WholeStageCodegen (1)
+                                                                                                                        Project [d_date_sk]
+                                                                                                                          Filter [d_year,d_date_sk]
+                                                                                                                            ColumnarToRow
+                                                                                                                              InputAdapter
+                                                                                                                                Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+                                                                                                          InputAdapter
+                                                                                                            ReusedExchange [d_date_sk] #9
+                                                                                                      InputAdapter
+                                                                                                        BroadcastExchange #10
+                                                                                                          WholeStageCodegen (8)
+                                                                                                            SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+                                                                                                              InputAdapter
+                                                                                                                WholeStageCodegen (3)
+                                                                                                                  Sort [i_brand_id,i_class_id,i_category_id]
                                                                                                                     InputAdapter
-                                                                                                                      WholeStageCodegen (5)
-                                                                                                                        Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                                                          InputAdapter
-                                                                                                                            Exchange [i_brand_id,i_class_id,i_category_id] #12
-                                                                                                                              WholeStageCodegen (4)
-                                                                                                                                Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
-                                                                                                                                  ColumnarToRow
-                                                                                                                                    InputAdapter
-                                                                                                                                      Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                                      Exchange [i_brand_id,i_class_id,i_category_id] #11
+                                                                                                                        WholeStageCodegen (2)
+                                                                                                                          Filter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                                            ColumnarToRow
+                                                                                                                              InputAdapter
+                                                                                                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                              InputAdapter
+                                                                                                                WholeStageCodegen (7)
+                                                                                                                  Sort [i_brand_id,i_class_id,i_category_id]
                                                                                                                     InputAdapter
-                                                                                                                      WholeStageCodegen (9)
-                                                                                                                        Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                                                          InputAdapter
-                                                                                                                            Exchange [i_brand_id,i_class_id,i_category_id] #13
-                                                                                                                              WholeStageCodegen (8)
-                                                                                                                                Project [i_brand_id,i_class_id,i_category_id]
-                                                                                                                                  BroadcastHashJoin [cs_item_sk,i_item_sk]
-                                                                                                                                    Project [cs_item_sk]
-                                                                                                                                      BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                                                                                                        Filter [cs_item_sk]
-                                                                                                                                          ColumnarToRow
-                                                                                                                                            InputAdapter
-                                                                                                                                              Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
-                                                                                                                                                ReusedSubquery [d_date_sk] #2
+                                                                                                                      Exchange [i_brand_id,i_class_id,i_category_id] #12
+                                                                                                                        WholeStageCodegen (6)
+                                                                                                                          Project [i_brand_id,i_class_id,i_category_id]
+                                                                                                                            BroadcastHashJoin [cs_item_sk,i_item_sk]
+                                                                                                                              Project [cs_item_sk]
+                                                                                                                                BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+                                                                                                                                  Filter [cs_item_sk]
+                                                                                                                                    ColumnarToRow
+                                                                                                                                      InputAdapter
+                                                                                                                                        Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+                                                                                                                                          ReusedSubquery [d_date_sk] #2
+                                                                                                                                  InputAdapter
+                                                                                                                                    ReusedExchange [d_date_sk] #9
+                                                                                                                              InputAdapter
+                                                                                                                                BroadcastExchange #13
+                                                                                                                                  WholeStageCodegen (5)
+                                                                                                                                    Filter [i_item_sk]
+                                                                                                                                      ColumnarToRow
                                                                                                                                         InputAdapter
-                                                                                                                                          ReusedExchange [d_date_sk] #10
-                                                                                                                                    InputAdapter
-                                                                                                                                      BroadcastExchange #14
-                                                                                                                                        WholeStageCodegen (7)
-                                                                                                                                          Filter [i_item_sk]
-                                                                                                                                            ColumnarToRow
-                                                                                                                                              InputAdapter
-                                                                                                                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                                                                                          Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                                            InputAdapter
+                                                                              WholeStageCodegen (15)
+                                                                                Sort [i_brand_id,i_class_id,i_category_id]
                                                                                   InputAdapter
-                                                                                    WholeStageCodegen (17)
-                                                                                      Sort [i_brand_id,i_class_id,i_category_id]
-                                                                                        InputAdapter
-                                                                                          Exchange [i_brand_id,i_class_id,i_category_id] #15
-                                                                                            WholeStageCodegen (16)
-                                                                                              Project [i_brand_id,i_class_id,i_category_id]
-                                                                                                BroadcastHashJoin [ws_item_sk,i_item_sk]
-                                                                                                  Project [ws_item_sk]
-                                                                                                    BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                                                                                      Filter [ws_item_sk]
-                                                                                                        ColumnarToRow
-                                                                                                          InputAdapter
-                                                                                                            Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
-                                                                                                              ReusedSubquery [d_date_sk] #2
-                                                                                                      InputAdapter
-                                                                                                        ReusedExchange [d_date_sk] #10
-                                                                                                  InputAdapter
-                                                                                                    ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14
+                                                                                    Exchange [i_brand_id,i_class_id,i_category_id] #14
+                                                                                      WholeStageCodegen (14)
+                                                                                        Project [i_brand_id,i_class_id,i_category_id]
+                                                                                          BroadcastHashJoin [ws_item_sk,i_item_sk]
+                                                                                            Project [ws_item_sk]
+                                                                                              BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+                                                                                                Filter [ws_item_sk]
+                                                                                                  ColumnarToRow
+                                                                                                    InputAdapter
+                                                                                                      Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+                                                                                                        ReusedSubquery [d_date_sk] #2
+                                                                                                InputAdapter
+                                                                                                  ReusedExchange [d_date_sk] #9
+                                                                                            InputAdapter
+                                                                                              ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
                                                         InputAdapter
-                                                          ReusedExchange [d_date_sk] #5
+                                                          ReusedExchange [d_date_sk] #4
                                                     InputAdapter
-                                                      BroadcastExchange #16
-                                                        WholeStageCodegen (42)
-                                                          SortMergeJoin [i_item_sk,ss_item_sk]
-                                                            InputAdapter
-                                                              WholeStageCodegen (23)
-                                                                Sort [i_item_sk]
-                                                                  InputAdapter
-                                                                    Exchange [i_item_sk] #17
-                                                                      WholeStageCodegen (22)
-                                                                        Filter [i_item_sk]
-                                                                          ColumnarToRow
-                                                                            InputAdapter
-                                                                              Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+                                                      BroadcastExchange #15
+                                                        WholeStageCodegen (36)
+                                                          BroadcastHashJoin [i_item_sk,ss_item_sk]
+                                                            Filter [i_item_sk]
+                                                              ColumnarToRow
+                                                                InputAdapter
+                                                                  Scan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
                                                             InputAdapter
-                                                              WholeStageCodegen (41)
-                                                                Sort [ss_item_sk]
-                                                                  InputAdapter
-                                                                    ReusedExchange [ss_item_sk] #6
-                                  WholeStageCodegen (88)
+                                                              ReusedExchange [ss_item_sk] #5
+                                  WholeStageCodegen (76)
                                     Filter [sales]
                                       ReusedSubquery [average_sales] #3
                                       HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
-                                          Exchange [i_brand_id,i_class_id,i_category_id] #20
-                                            WholeStageCodegen (87)
+                                          Exchange [i_brand_id,i_class_id,i_category_id] #18
+                                            WholeStageCodegen (75)
                                               HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                                                 Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id]
                                                   BroadcastHashJoin [cs_item_sk,i_item_sk]
                                                     Project [cs_item_sk,cs_quantity,cs_list_price]
                                                       BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                        SortMergeJoin [cs_item_sk,ss_item_sk]
-                                                          InputAdapter
-                                                            WholeStageCodegen (46)
-                                                              Sort [cs_item_sk]
-                                                                InputAdapter
-                                                                  Exchange [cs_item_sk] #21
-                                                                    WholeStageCodegen (45)
-                                                                      Filter [cs_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
-                                                                              ReusedSubquery [d_date_sk] #1
+                                                        BroadcastHashJoin [cs_item_sk,ss_item_sk]
+                                                          Filter [cs_item_sk]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+                                                                  ReusedSubquery [d_date_sk] #1
                                                           InputAdapter
-                                                            WholeStageCodegen (64)
-                                                              Sort [ss_item_sk]
-                                                                InputAdapter
-                                                                  ReusedExchange [ss_item_sk] #6
+                                                            ReusedExchange [ss_item_sk] #5
                                                         InputAdapter
-                                                          ReusedExchange [d_date_sk] #5
+                                                          ReusedExchange [d_date_sk] #4
                                                     InputAdapter
-                                                      ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16
-                                  WholeStageCodegen (132)
+                                                      ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
+                                  WholeStageCodegen (114)
                                     Filter [sales]
                                       ReusedSubquery [average_sales] #3
                                       HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
-                                          Exchange [i_brand_id,i_class_id,i_category_id] #22
-                                            WholeStageCodegen (131)
+                                          Exchange [i_brand_id,i_class_id,i_category_id] #19
+                                            WholeStageCodegen (113)
                                               HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count]
                                                 Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id]
                                                   BroadcastHashJoin [ws_item_sk,i_item_sk]
                                                     Project [ws_item_sk,ws_quantity,ws_list_price]
                                                       BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                                        SortMergeJoin [ws_item_sk,ss_item_sk]
-                                                          InputAdapter
-                                                            WholeStageCodegen (90)
-                                                              Sort [ws_item_sk]
-                                                                InputAdapter
-                                                                  Exchange [ws_item_sk] #23
-                                                                    WholeStageCodegen (89)
-                                                                      Filter [ws_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
-                                                                              ReusedSubquery [d_date_sk] #1
+                                                        BroadcastHashJoin [ws_item_sk,ss_item_sk]
+                                                          Filter [ws_item_sk]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+                                                                  ReusedSubquery [d_date_sk] #1
                                                           InputAdapter
-                                                            WholeStageCodegen (108)
-                                                              Sort [ss_item_sk]
-                                                                InputAdapter
-                                                                  ReusedExchange [ss_item_sk] #6
+                                                            ReusedExchange [ss_item_sk] #5
                                                         InputAdapter
-                                                          ReusedExchange [d_date_sk] #5
+                                                          ReusedExchange [d_date_sk] #4
                                                     InputAdapter
-                                                      ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16
-                  WholeStageCodegen (269)
+                                                      ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15
+                  WholeStageCodegen (233)
                     HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
                       InputAdapter
-                        Exchange [channel,i_brand_id,i_class_id] #24
-                          WholeStageCodegen (268)
+                        Exchange [channel,i_brand_id,i_class_id] #20
+                          WholeStageCodegen (232)
                             HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
                               HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
                                 InputAdapter
                                   ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2
-                  WholeStageCodegen (404)
+                  WholeStageCodegen (350)
                     HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
                       InputAdapter
-                        Exchange [channel,i_brand_id] #25
-                          WholeStageCodegen (403)
+                        Exchange [channel,i_brand_id] #21
+                          WholeStageCodegen (349)
                             HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
                               HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
                                 InputAdapter
                                   ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2
-                  WholeStageCodegen (539)
+                  WholeStageCodegen (467)
                     HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
                       InputAdapter
-                        Exchange [channel] #26
-                          WholeStageCodegen (538)
+                        Exchange [channel] #22
+                          WholeStageCodegen (466)
                             HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
                               HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
                                 InputAdapter
                                   ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2
-                  WholeStageCodegen (674)
+                  WholeStageCodegen (584)
                     HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum]
                       InputAdapter
-                        Exchange #27
-                          WholeStageCodegen (673)
+                        Exchange #23
+                          WholeStageCodegen (583)
                             HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum]
                               HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum]
                                 InputAdapter


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org